From 092f01e2c84407bdcb57487bf377d2dc501c1350 Mon Sep 17 00:00:00 2001 From: jiafu zhang Date: Thu, 21 Jan 2021 19:13:47 +0800 Subject: [PATCH 01/12] reconstruct project and add new shuffle-daos plugin Signed-off-by: jiafu zhang --- README.md | 38 +- pom.xml | 539 +++++++------- shuffle-daos/LICENSE | 202 +++++ shuffle-daos/README.md | 6 + shuffle-daos/dev/checkstyle.license | 22 + shuffle-daos/dev/checkstyle.xml | 195 +++++ shuffle-daos/images/shuffle.png | Bin 0 -> 25676 bytes shuffle-daos/pom.xml | 239 ++++++ shuffle-daos/scalastyle-config.xml | 392 ++++++++++ .../shuffle/daos/BoundThreadExecutors.java | 204 ++++++ .../apache/spark/shuffle/daos/DaosReader.java | 244 +++++++ .../spark/shuffle/daos/DaosShuffleIO.java | 284 +++++++ .../shuffle/daos/DaosShuffleInputStream.java | 690 ++++++++++++++++++ .../shuffle/daos/DaosShuffleOutputStream.java | 75 ++ .../apache/spark/shuffle/daos/DaosWriter.java | 690 ++++++++++++++++++ .../spark/shuffle/daos/LinkedTaskContext.java | 143 ++++ .../spark/shuffle/daos/TaskSubmitter.java | 304 ++++++++ .../shuffle/daos/DaosShuffleManager.scala | 181 +++++ .../shuffle/daos/DaosShuffleReader.scala | 129 ++++ .../shuffle/daos/DaosShuffleWriter.scala | 101 +++ .../shuffle/daos/MapPartitionsWriter.scala | 494 +++++++++++++ .../spark/shuffle/daos/PartitionOutput.scala | 117 +++ .../daos/ShufflePartitionIterator.scala | 243 ++++++ .../spark/shuffle/daos/SizeSampler.scala | 138 ++++ .../daos/SizeSamplerAppendOnlyMap.scala | 50 ++ .../shuffle/daos/SizeSamplerPairBuffer.scala | 100 +++ .../apache/spark/shuffle/daos/package.scala | 233 ++++++ .../spark/shuffle/daos/DaosShuffleIOTest.java | 106 +++ .../daos/DaosShuffleInputStreamTest.java | 394 ++++++++++ .../spark/shuffle/daos/DaosWriterTest.java | 156 ++++ .../shuffle/daos/DaosShuffleReaderSuite.scala | 212 ++++++ .../shuffle/daos/DaosShuffleWriterPerf.scala | 106 +++ .../shuffle/daos/DaosShuffleWriterSuite.scala | 108 +++ .../spark/shuffle/daos/SizeSamplerSuite.scala | 103 +++ .../apache/spark/shuffle/daos/SizeTest.scala | 45 ++ .../spark/shuffle/daos/TaskContextObj.scala | 37 + CHANGELOG.md => shuffle-hadoop/CHANGELOG.md | 0 shuffle-hadoop/README.md | 214 ++++++ {dev => shuffle-hadoop/dev}/checkstyle.xml | 0 .../dev}/post_results_to_PR.sh | 0 .../docs}/OAP-Developer-Guide.md | 0 .../docs}/OAP-Installation-Guide.md | 0 {docs => shuffle-hadoop/docs}/User-Guide.md | 0 {docs => shuffle-hadoop/docs}/index.md | 0 mkdocs.yml => shuffle-hadoop/mkdocs.yml | 0 shuffle-hadoop/pom.xml | 262 +++++++ .../scalastyle-config.xml | 0 .../shuffle/MyOneForOneBlockFetcher.java | 0 .../RemoteBypassMergeSortShuffleWriter.java | 0 .../spark/shuffle/sort/RemoteSpillInfo.java | 0 .../sort/RemoteUnsafeShuffleSorter.java | 0 .../sort/RemoteUnsafeShuffleWriter.java | 0 .../network/netty/MyNettyBlockRpcServer.scala | 0 .../netty/RemoteShuffleTransferService.scala | 0 .../apache/spark/scheduler/DAGScheduler.scala | 0 .../HadoopFileSegmentManagedBuffer.scala | 0 .../shuffle/remote/RemoteAggregator.scala | 0 .../remote/RemoteBlockObjectWriter.scala | 0 .../remote/RemoteShuffleBlockIterator.scala | 0 .../remote/RemoteShuffleBlockResolver.scala | 0 .../shuffle/remote/RemoteShuffleConf.scala | 0 .../shuffle/remote/RemoteShuffleManager.scala | 0 .../shuffle/remote/RemoteShuffleReader.scala | 0 .../shuffle/remote/RemoteShuffleUtils.scala | 0 .../shuffle/remote/RemoteShuffleWriter.scala | 0 .../RPartitionedAppendOnlyMap.scala | 0 .../collection/RPartitionedPairBuffer.scala | 0 .../RWritablePartitionedPairCollection.scala | 0 .../util/collection/RemoteAppendOnlyMap.scala | 0 .../spark/util/collection/RemoteSorter.scala | 0 .../sort/RemoteUnsafeShuffleWriterSuite.java | 0 .../RemoteShuffleBlockIteratorSuite.scala | 0 .../RemoteShuffleBlockObjectWriterSuite.scala | 0 .../RemoteShuffleBlockResolverSuite.scala | 0 .../remote/RemoteShuffleManagerSuite.scala | 0 .../apache/spark/shuffle/remote/package.scala | 0 .../collection/RemoteAppendOnlyMapSuite.scala | 0 .../util/collection/RemoteSorterSuite.scala | 0 .../spark/util/collection/package.scala | 0 79 files changed, 7511 insertions(+), 285 deletions(-) create mode 100644 shuffle-daos/LICENSE create mode 100644 shuffle-daos/README.md create mode 100644 shuffle-daos/dev/checkstyle.license create mode 100644 shuffle-daos/dev/checkstyle.xml create mode 100644 shuffle-daos/images/shuffle.png create mode 100644 shuffle-daos/pom.xml create mode 100644 shuffle-daos/scalastyle-config.xml create mode 100644 shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/BoundThreadExecutors.java create mode 100644 shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosReader.java create mode 100644 shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleIO.java create mode 100644 shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleInputStream.java create mode 100644 shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleOutputStream.java create mode 100644 shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosWriter.java create mode 100644 shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/LinkedTaskContext.java create mode 100644 shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/TaskSubmitter.java create mode 100644 shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/DaosShuffleManager.scala create mode 100644 shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/DaosShuffleReader.scala create mode 100644 shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/DaosShuffleWriter.scala create mode 100644 shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/MapPartitionsWriter.scala create mode 100644 shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/PartitionOutput.scala create mode 100644 shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/ShufflePartitionIterator.scala create mode 100644 shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/SizeSampler.scala create mode 100644 shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/SizeSamplerAppendOnlyMap.scala create mode 100644 shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/SizeSamplerPairBuffer.scala create mode 100644 shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/package.scala create mode 100644 shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleIOTest.java create mode 100644 shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleInputStreamTest.java create mode 100644 shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosWriterTest.java create mode 100644 shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleReaderSuite.scala create mode 100644 shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleWriterPerf.scala create mode 100644 shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleWriterSuite.scala create mode 100644 shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/SizeSamplerSuite.scala create mode 100644 shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/SizeTest.scala create mode 100644 shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/TaskContextObj.scala rename CHANGELOG.md => shuffle-hadoop/CHANGELOG.md (100%) create mode 100644 shuffle-hadoop/README.md rename {dev => shuffle-hadoop/dev}/checkstyle.xml (100%) rename {dev => shuffle-hadoop/dev}/post_results_to_PR.sh (100%) rename {docs => shuffle-hadoop/docs}/OAP-Developer-Guide.md (100%) rename {docs => shuffle-hadoop/docs}/OAP-Installation-Guide.md (100%) rename {docs => shuffle-hadoop/docs}/User-Guide.md (100%) rename {docs => shuffle-hadoop/docs}/index.md (100%) rename mkdocs.yml => shuffle-hadoop/mkdocs.yml (100%) create mode 100644 shuffle-hadoop/pom.xml rename scalastyle-config.xml => shuffle-hadoop/scalastyle-config.xml (100%) rename {src => shuffle-hadoop/src}/main/java/org/apache/spark/network/shuffle/MyOneForOneBlockFetcher.java (100%) rename {src => shuffle-hadoop/src}/main/java/org/apache/spark/shuffle/sort/RemoteBypassMergeSortShuffleWriter.java (100%) rename {src => shuffle-hadoop/src}/main/java/org/apache/spark/shuffle/sort/RemoteSpillInfo.java (100%) rename {src => shuffle-hadoop/src}/main/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleSorter.java (100%) rename {src => shuffle-hadoop/src}/main/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleWriter.java (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/network/netty/MyNettyBlockRpcServer.scala (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/network/netty/RemoteShuffleTransferService.scala (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/scheduler/DAGScheduler.scala (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/shuffle/remote/HadoopFileSegmentManagedBuffer.scala (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/shuffle/remote/RemoteAggregator.scala (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/shuffle/remote/RemoteBlockObjectWriter.scala (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockIterator.scala (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockResolver.scala (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleConf.scala (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleManager.scala (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleReader.scala (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleUtils.scala (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleWriter.scala (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/util/collection/RPartitionedAppendOnlyMap.scala (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/util/collection/RPartitionedPairBuffer.scala (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/util/collection/RWritablePartitionedPairCollection.scala (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/util/collection/RemoteAppendOnlyMap.scala (100%) rename {src => shuffle-hadoop/src}/main/scala/org/apache/spark/util/collection/RemoteSorter.scala (100%) rename {src => shuffle-hadoop/src}/test/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleWriterSuite.java (100%) rename {src => shuffle-hadoop/src}/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockIteratorSuite.scala (100%) rename {src => shuffle-hadoop/src}/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockObjectWriterSuite.scala (100%) rename {src => shuffle-hadoop/src}/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockResolverSuite.scala (100%) rename {src => shuffle-hadoop/src}/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleManagerSuite.scala (100%) rename {src => shuffle-hadoop/src}/test/scala/org/apache/spark/shuffle/remote/package.scala (100%) rename {src => shuffle-hadoop/src}/test/scala/org/apache/spark/util/collection/RemoteAppendOnlyMapSuite.scala (100%) rename {src => shuffle-hadoop/src}/test/scala/org/apache/spark/util/collection/RemoteSorterSuite.scala (100%) rename {src => shuffle-hadoop/src}/test/scala/org/apache/spark/util/collection/package.scala (100%) diff --git a/README.md b/README.md index fcdb839c..26e67b66 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,4 @@ -# Remote Shuffle +# Remote Shuffle Plugins ## Online Documentation @@ -9,10 +9,18 @@ You can find the all the PMem Spill documents on the [project web page](https:// - [User Guide](#userguide) ## Introduction -Remote Shuffle is a Spark* ShuffleManager plugin, shuffling data through a remote Hadoop-compatible file system, as opposed to vanilla Spark's local-disks. +Remote Shuffle is a Spark* ShuffleManager plugin, shuffling data through a remote datastore, as opposed to vanilla Spark's local-disks. This is an essential part of enabling Spark on disaggregated compute and storage architecture. +There are two shuffle plugins in this project. +- shuffle-hadoop, A remote shuffle plugin based Hadoop filesystem. + This plugin can work with any remote filesystems compatible with Hadoop, like HDFS, AWS S3 and [DAOS](https://github.com/daos-stack/daos). +- shuffle-daos + Different from the above general plugin based on Hadoop Filesystem interface, this plugin bases on DAOS Object API. + Thanks to DAOS Distribution Key and Attribute Key, we can improve performance by constructing shuffle output like + below. + ![](./shuffle-daos/images/shuffle.png) ### Installation We have provided a Conda package which will automatically install dependencies needed by OAP, you can refer to [OAP-Installation-Guide](./docs/OAP-Installation-Guide.md) for more information. If you have finished [OAP-Installation-Guide](./docs/OAP-Installation-Guide.md), you can find compiled OAP jars in `$HOME/miniconda2/envs/oapenv/oap_jars/`. @@ -23,12 +31,12 @@ We have provided a Conda package which will automatically install dependencies n We have provided a Conda package which will automatically install dependencies needed by OAP, you can refer to [OAP-Installation-Guide](./docs/OAP-Installation-Guide.md) for more information. If you have finished [OAP-Installation-Guide](./docs/OAP-Installation-Guide.md), you can find compiled remote shuffle jars under `$HOME/miniconda2/envs/oapenv/oap_jars`. Then just skip this section and jump to [User Guide](#user-guide). -Build this module using the following command in `OAP/oap-shuffle/remote-shuffle` folder. This file needs to be deployed on every compute node that runs Spark. Manually place it on all nodes or let resource manager do the work. +Build using the following command in `OAP/remote-shuffle` folder. This file needs to be deployed on every compute node that runs Spark. Manually place it on all nodes or let resource manager do the work. ``` mvn -DskipTests clean package ``` -## User Guide +## User Guide (shuffle-hadoop) ### Enable Remote Shuffle Add the `.jar` files to the classpath of Spark driver and executors: Put the @@ -212,3 +220,25 @@ shuffle reader: shuffle storage daos://default:1 shuffle folder: /tmp/shuffle ``` + +## User Guide (shuffle-daos) + +Most of [User Guide (shuffle-hadoop)](#g1) can be applied to shuffle-daos. We'll not repeat them here. Just show +differences here. + +### Shuffle Manager + +``` + spark.shuffle.manager org.apache.spark.shuffle.daos.DaosShuffleManager +``` + +### Classpath + +``` + spark.executor.extraClassPath $HOME/miniconda2/envs/oapenv/oap_jars/daos-java-.jar + $HOME/miniconda2/envs/oapenv/oap_jars/hadoop-daos-.jar + $HOME/miniconda2/envs/oapenv/oap_jars/shuffle-daos-.jar + spark.driver.extraClassPath $HOME/miniconda2/envs/oapenv/oap_jars/daos-java-.jar + $HOME/miniconda2/envs/oapenv/oap_jars/hadoop-daos-.jar + $HOME/miniconda2/envs/oapenv/oap_jars/shuffle-daos-.jar +``` \ No newline at end of file diff --git a/pom.xml b/pom.xml index 7345b02a..79d831e1 100644 --- a/pom.xml +++ b/pom.xml @@ -1,289 +1,266 @@ - + + 4.0.0 - - 4.0.0 + com.intel.oap + remote-shuffle-parent + 1.1.0 + OAP Remote Shuffle Parent POM + pom - com.intel.oap - oap-remote-shuffle - 1.1.0 - OAP Project Remote Shuffle - jar + + 2.12.10 + 2.12 + 1.8 + ${java.version} + ${java.version} + 3.0.0 + - - - Chenzhao Guo - chenzhao.guo@intel.com - - + + + + + org.codehaus.mojo + build-helper-maven-plugin + 3.0.0 + + + add-source + validate + + add-source + + + + src/main/scala + + + + + add-test-source + validate + + add-test-source + + + + src/test/scala + + + + + + + net.alchim31.maven + scala-maven-plugin + 4.3.0 + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile-first + process-test-resources + + testCompile + + + + attach-scaladocs + verify + + doc-jar + + + + + ${scala.version} + true + true + incremental + + -unchecked + -deprecation + -feature + -explaintypes + -target:jvm-1.8 + + + -Xms1024m + -Xmx1024m + -XX:ReservedCodeCacheSize=1g + + + -source + ${java.version} + -target + ${java.version} + -Xlint:all,-serial,-path,-try + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.8.1 + + + compile + + compile + + + + + ${java.version} + ${java.version} + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.7 + + + org.scalatest + scalatest-maven-plugin + 2.0.0 + + + test + + test + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 3.0.0 + + false + true + + src/main/java + src/main/scala + + + src/test/java + src/test/scala + + dev/checkstyle.xml + dev/checkstyle.license + ${basedir}/target/checkstyle-output.xml + ${java.version} + ${java.version} + + + + compile + + check + + + + + + org.scalastyle + scalastyle-maven-plugin + 1.0.0 + + false + true + false + true + + src/main/scala + + + src/test/scala + + scalastyle-config.xml + target/scalastyle-output.xml + UTF-8 + UTF-8 + + + + compile + + check + + + + + + + - - 2.12.10 - 2.12 - 3.0.0 - 1.8 - UTF-8 - UTF-8 - + + + + org.scala-lang + scala-library + ${scala.version} + + + org.apache.spark + spark-core_2.12 + ${spark.version} + + + junit + junit + 4.13.1 + test + + + org.scalatest + scalatest_2.12 + 3.0.8 + test + + + org.apache.spark + spark-core_2.12 + ${spark.version} + tests + test-jar + test + + + org.mockito + mockito-core + 2.23.4 + test + + + org.powermock + powermock-module-junit4 + 2.0.2 + test + + + org.powermock + powermock-api-mockito2 + 2.0.2 + test + + + - - - org.scala-lang - scala-library - ${scala.version} - - - org.apache.spark - spark-core_2.12 - ${spark.version} - - - org.apache.spark - spark-core_2.12 - ${spark.version} - test-jar - test - - - org.apache.spark - spark-sql_2.12 - 3.0.0 - test - - - org.apache.hadoop - hadoop-client - 2.7.4 - - - org.scalatest - scalatest_${scala.binary.version} - test - 3.0.3 - - - junit - junit - 4.12 - test - - - org.hamcrest - hamcrest-core - 1.3 - test - - - org.hamcrest - hamcrest-library - 1.3 - test - - - org.mockito - mockito-core - 2.23.4 - test - - - org.mock-server - mockserver-netty - 5.11.0 - - - org.mock-server - mockserver-client-java - 5.11.0 - - - org.eclipse.jetty - jetty-servlet - 9.4.12.v20180830 - test - - - commons-cli - commons-cli - 1.4 - test - - - - - - - net.alchim31.maven - scala-maven-plugin - 3.4.4 - - - scala-compile-first - process-resources - - add-source - compile - - - - scala-test-compile - process-test-resources - - testCompile - - - - - - org.apache.maven.plugins - maven-compiler-plugin - 2.0.2 - - ${java.version} - ${java.version} - - - - compile - - compile - - - - - - org.apache.maven.plugins - maven-surefire-plugin - 2.7 - - - **/Test*.java - **/*Test.java - **/*TestCase.java - **/*Suite.java - - - 1 - ${project.basedir} - - - 1 - ${scala.binary.version} - - - - - org.scalatest - scalatest-maven-plugin - 1.0 - - . - WDF TestSuite.txt - - 1 - ${project.basedir} - - - 1 - ${scala.binary.version} - - - - - test - - test - - - - - - org.apache.maven.plugins - maven-jar-plugin - 3.0.2 - - - - test-jar - - - - - - org.apache.maven.plugins - maven-checkstyle-plugin - 3.0.0 - - false - true - - src/main/java - src/main/scala - - - src/test/java - src/test/scala - - dev/checkstyle.xml - ${basedir}/target/checkstyle-output.xml - ${project.build.sourceEncoding} - ${project.reporting.outputEncoding} - - - - compile - - check - - - - - - - + + shuffle-hadoop + shuffle-daos + diff --git a/shuffle-daos/LICENSE b/shuffle-daos/LICENSE new file mode 100644 index 00000000..57bc88a1 --- /dev/null +++ b/shuffle-daos/LICENSE @@ -0,0 +1,202 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. + diff --git a/shuffle-daos/README.md b/shuffle-daos/README.md new file mode 100644 index 00000000..57b07bb4 --- /dev/null +++ b/shuffle-daos/README.md @@ -0,0 +1,6 @@ +# Remote Shuffle Based on DAOS Object API + +A remote shuffle plugin based on DAOS Object API. You can find DAOS and DAOS Java Wrapper in https://github.com/daos-stack/daos and https://github.com/daos-stack/daos/tree/master/src/client/java. +Thanks to DAOS, the plugin is espacially good for small shuffle block, such as around 200KB. + +# TO BE CONTINUED diff --git a/shuffle-daos/dev/checkstyle.license b/shuffle-daos/dev/checkstyle.license new file mode 100644 index 00000000..b2188bea --- /dev/null +++ b/shuffle-daos/dev/checkstyle.license @@ -0,0 +1,22 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ \ No newline at end of file diff --git a/shuffle-daos/dev/checkstyle.xml b/shuffle-daos/dev/checkstyle.xml new file mode 100644 index 00000000..ca7dd9eb --- /dev/null +++ b/shuffle-daos/dev/checkstyle.xml @@ -0,0 +1,195 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/shuffle-daos/images/shuffle.png b/shuffle-daos/images/shuffle.png new file mode 100644 index 0000000000000000000000000000000000000000..e37d7c402ce042d0e8a818c33de3cea2975ee23e GIT binary patch literal 25676 zcmeFYWmMEr_b5ET5JM^>f^-fgDJ5N!10o^aQqnC*j)b6ubcYHE(hO44(g*{H$j~AJ z(%o=>gFesy{qU}JzuXV^!*wl(nG^f$ea=3+Pn4FXG6^9yAqWH_QB_gY27&M(AP_c! z00(H9ChyY${$P1(E6an*hiF%UA7DE<4LJ~~Dvs#f5*PTrpas``0Q|Mc9R@xPQYU~< z;Ga>(IPhtbIbxDI27ED0|NghpC>_&oobk;#^V@aHH=}gS0OQQ>#@ADfGslfH#(?j} zS=SJ<#?^w#{|>nR9LEgEyatkm`C@YY^Ir=FEYR`|Gd*+M=szQQy+6shXDN8)Df+xq z@+(paC{Yb6R|~FC53SOSsMm`6{ODEdqgQP@v7NecUApl-j}!Y1QicrEMgSmgWy)7= zpz1_4DX_edm$s%3=mwOLnU7ymN>)wFz{1w=$fcX$N6+=%Di1H@pKUP0%PX+|CLaHjt={_^tXb!aAm95Fe z7v!srQP6NTNuW|r(P_A^m-T7#Sd#r=LSkJ`{N9C(mYuC#xY_M^S!~twav3K`Slcr@ z-i>k}UvNYRAIKZ!fdirjnez#>N0$$$ZYl=x3S!aNHvY8zAf~fHpuy}~ z(v$~+gW4D!X2D7Ab;#NR5@9*nr$4r#MAC}rOUl(+F)3uB;|}ZdoeEf`DcReEzKEfl zu1Gd^gOOREw=fk~rnmUctsO~mvELIGUg#-q)z->$rG*Ts<7X5&5n779A$+lk_=_$H z^fh+Nw?fHM*gQMO1N`&MDD0NjBwSfCG)d6Y;GNuzHa)J}=y0cdfff=0(7;aQQ`y!f zz4v&bHbP6f!!M(!mEW6l(Fc9(brC$jb8O#)AAL>bWB7dek~8yiZdPPNp+&xw8p?xd z*f4W+3k&MB-l=n1;8qsG;Ei$_0iQn&O_0)8zLlaf3HG3QGBUZI_Nf=44aJr zR*`#)c=B2f*R`7Y4WsJu+rv1|IWysYnk?|?_zIR-V@LiaVMFYcoDoTMge#9 zqyJQ9TaWb4qL;KduJ7tDZi^84q;d_+V$Y32F;6sgZ12vh!ZUbVMFgY-8w%oAF%rcR z#ZjH$k=e*w=7?ecv|H&);jym482IImz_!G;-sh}G6k7aPb)p5u*`IV)wsan5(5As1w7S4yV2^urI(nHAd_uLrVV7YsS1wCmG^3SI82bISx z&wnC&(q<(xm(^kMp1q9WQu_pOH&7ZVL~ism1V-fI_Qo5szI6YHz{&Y-Y`x)%gLRFJ zKCQ_jDpa65t7uj6-0?wc(LkCjq@a1-^u0;dOPJ47!XS#=R4%45l4i~chED-!BZw_d zo(zk|U)J|<2EhIVFtNXPel?WxJWU+ZvR$Dp;5CtqFPeO!)bQIs03RAk^avEv4V|A2e0 zrBfZTbLww7*K76=C3bc>-I=)$ZY07W&|qorx!2-AQcK;2oBXV}3 z-5nIdY9z)Cma~6BomUU=^bK?#9tSUom^@7616DK<=mj{R>C z|7K<@?;S6#obY1(Rg{~Ot*uuz5P%!om~No6f&B#`iRGL204WiYB4_VQp9TV<1PL~V zaG-$cgO@+!2Mm=RU_?rg!BKMi-P81MgF&GCT7u%3-$G@FH!g5My`c#seijJQ(LtGl z{9tF2Z8C7szY!ML{CEF1Utjqsyudy^*e3|{Sq_DW6TmG-zTgwCOpLKyBe=rIA6ihn zwZKr(v9;COM97b2kCm49q78gItqmUzG6KOcBXD8cSkRE$A=mRmApvdRJX`=HY^Z?+ zHpBnfDL2SYX@qbA>MgK|;PW^z#@QA#FCrnF|4+;1y3d@OTw)uT&a$xePFfo_^f&}U zAc(ai^4-);fL~7e+0ZS=f6XuT1+W@Gba*6KP_M9McvS@CE`KOlTFA31W=84iS_d}~ zV7vu%-Dm43_VtwdhtStOWMnmyi+g)}18Zw*xnvMZBJ|4)U?&K^gT12m2%usB)e6FL zOcSlm&8@)!K|z6mfmi3}>p?*+t*tH6GSbqsbF;ITpB?MhePk+%9`FlCxr8bJ_%1vm zI&?WY`MYly!tH4ZQcM17hkQxvoB*s~mFla)C+33(6%~AZ!*Ovnlk1^o>~?+gyjdmB zEQB&>-Wku|;BQY}u+MXge-{*}ny+|}dO@2^4t%GwVq8*=#8_Zlv*&hRtW zf;lEUtAmT4JZ!i;STanwz+rnXrSM=qd@Rmc_OOyZ z!qx&25l>w4NY*ls!lY#3k4>D((Ye+_ZK||+uE!3Gz6?3};U}@*%LlGB{UMF+!WIZe zw?%$c8h4XB6H2trt!L^RCc(4Pla3&!*(;`vQC9lLaw4#*qKME)k{Az zp5!YZWMgH_BVHYwM5n$wR{DtSxdC0VA%$B2ysvwja-Tp9k95i3<;wyh*Y(T&ckM0I zF_{)#Kdpx}B#o$`_039fe!`y}8Y-E3^Ra&<0tMx|Zt>kQ(+>DZxkGYqNY~dA!`q#K z1OAp<@^aYu@oO}zSL&R< z@LkG@pGnxzCl1}<;5Wn-OBsZRE!h$G&ih^+yA&Qrzm{R}CW5)Ofu)14rdq#q(+9H* zIeT2Vm%AXozzy-UW+0B(O8*dk-oixpNLbUZ--nT&9{hBr} zqZTpYp@D16XYHT8FpLGOyaw3WdeM3{k<-;#t5F&6c*btM9++zdfa@)L;F#!URhN2% zWZ_Zq0yFrRXv%hIHLQn1s)ei(j^<)s6xX-w_#*HmGx%h@V$!Fa4sASD^RY}001#?& z`Ow5{F0E``m1q=J1nROz+<=LZ!cnZ(vPW)i_iSR|=#8A-%wm7O#)&Bxjw7A6Y{PU^ z8wq71HO|GiZ#h1kPeLJoaG*kv>r>jUJHNc|KhwNA@eQY(7Omdg;Y-(%G{C|h5~C{5 zltA2a!y%qxU*z3gdT5^a3YZ@wP5_Y?88DFR`flf}T=b)l>3yW%bOkd7R{xT^Gu}^4 z%gA~;qj-_>MKR?`aHw6H06ZQSW>0=iSG=p=;M(!f>{)fjmbZd2J0bb6%i#W{r1cXY z=1uut^Kn-df~0wM(MWO|=sfSJP8(R`uMjviroY-|oLb%*HuIu15k(&Pg5E&uVQnx1 z=I5EH5`MAQ#__kyS2-R|f~%Hg(-?{VvnTm;yH4ylT4v9yv7tq>z$ zNZIoSb8`il1v8!fl0Z6wa<*LWr1<4-r{$?$%T$2x;$#jrN9$%7PG)S%snM(a*j7AK zR@j$)%~+DA!croJir)!{65sQQiR4%`atzD%w1EvR5U|keE3|JMbXip!I(g*=k56d-D(LH8Y*mZoHN?rM1wNrhkYUZp?`@!b3Hjw z7qy+EV{q^pM-2&)x=#(K1X`Toak%gCrTcQA>m%s>zmdRtCm_4r<6ow1RyB1bIE8Vb zV*3H9ASlN*Q@lAznUnFh-bZh|vCSskxX3EVqiF*ZUZ4Jx)}>*xnfP++nLms`pVw`1 z)J+R)+-olOnA@HB=<-@=@hEr5htlgla{00yJn!?LslQs5+&|SkI~*T>V{1z4OmBoI zjtkGbW)9^`ZZJb{OmOgJW~)%>w06~ZSyo);;M-@5&bnO{^D}0 zb*U%UMA+iIW$K8|3-{W{c=cBFH%7NpW5c!|ou-(#dMm8XfARY3dOsL@N;V>s#{?*u zC09S0Qdz(A%Ri5OvE-Z9>_O%ffop;syo~sO{D8t39#FhC3!ZKKg&pQIG4d-wZ@+9H zuReqpP;usI0Vz#US6; zAc-;JU$`2CE;=76*SUZIdNJNrMYp=w6O=q5ZjZ zw%9m>`wwZc7EMyRdXN9E9c#@pn6z`YsV7?TAN=C*=YahzzyII0d)HSgv6oKH*3|wB zMv*?ag!N=1x9(#-=pTlCAA=V%n1jnly~WBVJ~`fUU1q!D4jun#DSL{wPgqyrZ*w;K|G ze}bZyxjbPex`A&yO%!9e`nv&UfUXv(U1Gg94-_hii@!afcx9Rz3XwI>-xm3;>iaUy zXY7Z}kPXb>(e_aX_}$LKB}fE1JFR>N6|~$R(lsY0)OTq&EY#lL0w>HU&}uohQhbRW z_`VL{#tN+vm#>d~pNU45WXR@Csrm6&$bIzDg@xM^qIGtM}v z+4TkFknUG+Kr`Qt=St?J>(`%*UT^DEYpIsgctFpkI@WW{3m@9|Z zR}vHYG$K><^ue5te3CpT%gvtENI>w^=8v7$nr`{49XJ^J8$S;b9JH65AqaMcD1 zKA9S)Rn6^x!H||Wg_K||oWm(%{ai==g2<&<4;nVr^+d>j)Jb!}=~LoJ*lu9U<9NJh zo6kIm%nx{$kOAN*M&2NeX2JHAJyTIGLB}Ag#@`Ch4=>~oQfES`2j$B*##~rAxmXeL zj{c5`+gCGYvtzN}E>{So_jR*F!WK@3$IkXIrL}>zRc^Y(z#&^qyE9sn>Xie|3MWB` zf{RU#tS*q8z{?t}p&yH$b7FLI%p7yfKlq^Er~r{7A~jmNgUSiq<5AG%-VPor)A)*} zX1&tI*cpj`Hd+HafBPo)YiMFc_kdfjB$G?)brnzX;BMR;Tl^zvvszll)Ksp;42dvw z-N2E}eA0yrd2#ueNMM{Fbj8<>CM3#xp=l6L%sQ^+q_|5tSD-X7ZG6&}AfTdDwkN6rY_J@MMy=-2Ih-v3m z`>n6G!VQDqz&m0#tu)5Xb1G>?uxCc$mQ<5lyu$65kIB_xZ8bfK{8plAN$?G7Dv$j^ z@ViKyixTB$_FnG|JM%9}ne%uqvh9lXkINO(F2~;xg~h0@2== z>op76zm+!s_=ooGyX4Lv_C0eTVNNGfAzLpmiL)Z8V}q;_8Cg9q)zq6WJb+Ab^;Zm zF^1B5vf`U+2*Tq#`^{ssDNzfAael#TqR}_Au*{92w`s?c$FVEJZJZY_@{O0qlX;(D z1S=1py;Uq9wp#=HD7c$zEj+W*JP34vzGeA*l{=drqN`j#VaP_aP0_EwyG$H zn09Z!GCIvw>uGK3X{*YwR)EaD&wpoqDo?K)LWm+HZmobTs_(ja4@t9uu2GrZn?i58|_9q6zVxr{2oTG-PRgE?ueWlV?vfql&%}*j}{**4dQp$G_8+ zzdTfJh7Fx2H13D(t0xGWR=TmX%{31-=^7QMsm-x%J$>WV5bL1>pMbHFq5DXZ`sT@J zI40d(SQg#dbANP3@VoU#)SatR_x@3R#M1TQY?FDsM3?07y}8!rv(pk>h@S@5sZ90u z)fZ;h?<_4tPOE;Gm!FYMc~z*OGx3?Zs;}=ojd4(Kt*qNjZ6-tV0?{6Pd2jQK2WP=* zR_1TyDWAr+=#v0kx*!<)U45qQnVMr#>T1#ZJG{vACJSt?k%Z4hRkTm{(utG$;$h-l zhe-|loT!_nIiq?SGVu7=R!y05M+xJR2_?#aD9Paq@TX#poGArWd5SbCX@f^1{Kk%Y zbYU7PGp#@Ls!r_?r+_t7(Ns5Jg3_{&o0H-83b3P&tlx<2=%An|vr_kbigef4-2c#| z@?RrZKzo!i0^{NscueG&_}B9HTNtVxd_p<=~U|Kj5T17KzyhF0N(`>d;om zG^2kZzn1OrEV$iDV|Xnsjfpf%oDg1V=hF>XCx>RidAm7E^3TtAW+MocR_}Tp5d&gV zs9}npM{|l(_r8iw)f{+o&A)tw>x5=n8+zB|d4$yR4SY>H99Zx6v1btX@cP-BQ=HIE z^kQ7yV@27}ge}V^b&PQ6Gu?8|`km=9`n!J&#zC5->JH1B8&zx>eu+aX<_%6`98fO< zlzEnnfGJM?ObWq=$EH*)ZmDhHx(+tql3PFCXKD2AzjtoXg=)2=m-s4*R+mxZzRms+!pF~B}nJg0}b1! z@?)aoSOlb$=v(;kD|b(ShRrrktv5t`<#xGt)35yuTXBGSOBr!k8frN-&ykT;5#8dQ-Xtg|P8w2v#Ktr5SwZ!K?=wviU6LR0 z71c>Psh#;ETsDu)+@gIi@y8|nT1xp~SK~lz>77Fdu9KiPoFdr=-7@SjZ)w$yO~inF zrhW<7_ls#dD8yJ`Dmzf(!(yQR$A;f;QJ=W=5NAMOcAh$`#!VyCA3A1Aj&>iGps4=|%Jn8Sps0L&-t4$&|>O);&Z!n3+Le{Z(}49DaGIBKnnM7Hr!q~`Oxo0H|n{w zbJT2(0j@aKNq7rz9JKBAjf*{8YCcwy{$fIl4}kNfh&)SAB6G^^FgZxVVddJp2XaCZ zCqOo!o?k^_56wiQ^ zLH!-FrU>P<`+(DY=^>8%SgxC1I7wnR$|IxLIxK>9k)Z@6C|4%;G_;OCr0CVBy~g(Q zDkcjh1~$9y-H9b`tV~nuMA-{nv*OVL{U)12q+II{uzKT{-!pwMALngkRld2$UGt!^ zu4k~~j}WEMoDk9}nAbte=)loQ%216LEbj}?at7?sd4*&Z+;^z@sixv?Mw?5Y^4AX6 z%nPwL3~MXo1fw&s_W7>ZWJXIB50bf#R`sY2j}IT6K8VW7ork4`AC7s@NSbe`d;A`< zMCau~wv4hMS!$!aPcm02eImUWT~(9^f_|*JAyjz}7<&43cvHf| zI`G@VJ*-xvyzFysDcBEvi9?m_w(lIWqyUR$j`mk#BNWaGBn7gy9Si#5{>kb=r)-8P zd^0DAEOIZ!hLfl-m8wwwwd-W2|Iaa0&3eMdlUr^^MW!y@bKubZO@DY>LKN&>A6I_r zFY{G_(6bMQ$HY-0ilKKgo4>TDAezw?;p1Q>uHK9}Xg!_#7sSGlf#lC7b~<=*PU5-m z|7A3B+R_b4-}X$ryJ_Rw(v})21*9|UX@UtW_S0o$p5lGS7k}Lh+|}u{ipPav-%<3f zTm3V*!jv3)nT$S$$+?>OYk{Y#abpw>%fA_e8gA4myyuy_spj4BX?-lB2fQEa;Q8<} zc^u9-L;OjC29u_&7ua~sudZOMoMH2!N!v!GSzF|{ZLojX8)Sw?f0qAhheSd&g7r+Y z7((tRIK$R4c($U{4)%yo2rifa_)|v_JSl)@fw{b84gm|)Fen-blT2-as%-;n_V~W6F;qRxPOD?P1bZ%mc)?(Piph%Zk`Ike7cs^WnZ${G7k|%3z6kzj7{F%3@4Q&7_oH|?Drm~#GI!j(`dO~oQqZGKWPy3U$@$pgxrNA*M-8zr z^E2wmEXeT@f2v9C*p}NZH|bn7a!4mwGxZZgT8)t97_VuDozMcjEg2D0 zhRaGM?$+{qr*}FivH4ZfoGTPA7UhXv-cw3^RQa-~OaHb;{K<#gemsh97=^%#I<7=` z01C9#)~-yqZ&kRfPzO!0;Vl0~V=bCBaA?$F@rJXXQcB-NR#H1#pE1X{-CXaIH@*}P zAGv+edH+vtk93(p&UUF=m;|RR`?0+NZVM{tBJ*lFcg2>jdR33F#^F`w`U!JTL+){V zEx}G_x%g**1=mvQS2wV(~Td2mE8 z>=$pW0HG0(spTg}Hv>6N5)tF@N~jwBk1HfaH@hIWoU1)t}BiNL6nDLMzdM$E|{J!=jy2o3XDO6IB% z6rck_rdS}lU%QsLv}o_gC0|r(nlLE63J6%~ZqTxyFv(j#IfKsN>i;}?x8B^QT5}lC z6;ux2MF1T0xcHWNP4-TunmCXGqop<$2y)|^Z#*E`oO_WJ+iJIwM1|&W+Cnw zY|yP^F=bp}I|?vCqNh(Zb?2?;VJgIlPLz%2xWMLg>Sl^^M4A@w?xKQDE@ge?)QP9T z%-AsE{WoI(wzP$pmDr}m)ln!VUpZsqS#Z{Pk?~m*HGqgR)z!DB?^P)|99fY)TTS{i zU=0sS7*&i_V(%7)P+IFxAKjl{1r_v1$YUs4(1ceCDh26d%ta32+vRGC}IC zIQ9t=0_{c`H%ghU6TT|?DD9DKIp?m=+P4V9%y5CFdi=`8OBtJF4PWG6V7NtSATW*b zrABXSQI&vY8WqbCIZv(V6 z48$22yy!#+Ow;D0crVzFnW6697S`mKwb3d2D?-%^V(bvOQX)woU{P%d2&|$Wa5-{? zTU_PI>fdb~f5a8TKTx4N>101V>-f-}B$odf)W8Z-AFjIo^}V~udSUnAD&?wN&ifJ| zlxM{1VVLt2zr1v$P{h#4s^+)3dQI{S>F!B6O{CxPkZGML4O(#2Y0gxjJFjM-o(#%* z*zdUZbbf7`B2`$<%9>27JI}_Xh{83k*ZGf9j-KyFyrM{oTlytVXt3ny-R!i@;Sl6(H!AW0kkJEG~{u<+6SJeZzic0pBDSmB@}f6$MX<5 zlysZJaTa{`#+FQ`>D7c`i|gjwT%drgKctM2;4?!!&fO=7h6601I>2Y-kDbM!qN{%1 zoXM&Dz3d2Bex$rh>-5u|q{6{ISc>t~vnCo^^ZLTi+BP&WH|SX4x=&N|hQb)N^{dU7 zP5fA7?DJ}J3Bxw+5|1vI)2lz6t1=B{SiDIK z`&fZSO+sQxxYLpWO1r@OnhfTA!>kC?u0y8A-(x)S3({~|hn$Lz4wI|QCi)s!UhWDz zNZ-EXUbzN^oy?aAb|{2q7lK^cc$|1+TM&W&o3aNKX;Z{#yr-{QE~T^_xDo02f16H*P=I1LBc5~>1vUH1mF6JL}S#Msp z2pf}F4+(|Zo+Jxny{W%h97{32Y=O|3{sG>fE7BtR5b>2OxBOSRHj7>u`{`8`q!%>R zXWt?0kp@CW>uPdJjo1zr!~`(vGo3BiJb-@Ss!wguDHxL}ogZgM9K!iO=^;8d6u37t ze51~+pxYHE=$MDzaF*{lKr)^tGsKnVZLW?>I0dlU!HLhuQ(?UvbPEO4d2AYa4FDi%%ehzwmMrL%coq?jg{V@Lrd`zM@Qy}!~ z`Q#r%tGD-!A~ii(pzl+))u)8G_edXA*Ijk6m&Ej}S$w=2!Ih%qS<%8!fP^@|&ysTY zmfgI?Oqvl=t*^GJwtpCqY#TVA%oT{VK}0effEJt?GRABe^-gLnTbZESG6GW@D(Rhc zW9oP=h|9ex{rx>0hH2Q7k8eGOGnyJD?Xfoo{xBEZw5`@Ii2Kz4rO8NkyZI=>6;G9q za2%{-+|)m)+HjSyF_FWtp)ZIf%KX8EmcQM>0&#ETGg<}%Ea}O#^G5!pWW(m*3mBvR zli$j#x5F)QeG1eVoRgy>BY{GZfq}K$x}&4v^GaLwjJu8T#dam8UdEaMk@zi`>v*s$=wvV```*OiQ_5VyaQaWr5nX zc(kXi{|hEko}T>$%!+M@5)oxzGAg2f$8y}RVoh4737vgVEZv(&hlBQU(TlCqG z`r8L%q85YfQ3@WmqL*YDuA@y%O`X>M!$U!#f@F1EP~$U=rWPL&tSH%EnYoDx+D%AC z=YSU?{jeW%aX&*KnmEub2%KN#*w$uBR92NkgO?2o_Nbw8O@Q7t&Ei^kKv{mD3t=2C zvgBJ#Tv{9S0OrQ**4cHeI<%0{H^v)&mtXE)8`usTN}N#Sbz56ao|osOe(Gq%)J8|4XX_K?tzs)nG%y-2T# z%fqiE^dL&|tACU}FP_TW%?7OT0`*Ur4KtmRY^jYgyiH~` zx`m&;VES=J%lwaLi2DyEjT$?F3O}GW1_tD#q^o5otHK@AGLxd5Vc8FD-hUFt(#N;A z%g9XBhFz(cyV<|4B42KLbwoE7P802q4M({H`pnO(pnv%V(FB1gCp5jA$>nUanruD( zbqwq2c~;Wyho*1C*GYB7OsjB4Jwmt}kPUzX)4WHMxyX1-tXChT{}?v(w*!im0IPzI zy+*qrc!9cr(uO%Zd~+5k03(Hi8Z}%B`X3H=2**k%MKuKTx6^^3UV!oBmC>P=+%--p zxStQ-Mpd7IVJl()9fE)*)$?iZhcZLIV)=`c^1ndD;eG}E9Ua1Vv|vCz-Zko`0o;1@ zw2wmsM*u^GH=&^f0g<)asnBgOuZNNKD*{=^#@>mRIT9GibP1)FTrt=92H)oGKpWWf zmuKDCBT68pm`Q7xHz-pI`r8}^m+igV3ZTHMtY31zUHq^4ONOio2pYocVTB>X3YPVDv;r`mXBp)=t>3YX0Px_zGJynk>JZn|zfQ-uQj(OGmcGZw$5%0!D-%4M zc64+^_7f-*QEQ4AwSrXFQsZLzWCYc&bNzt#0O|%cAbcZ7CbnfSB-a82|K9?_rNH+wL zKto~$&Mw~>_xN27DU$+Awr`me0i){+J%-jZ`aFxXG_+Kfvie+8`}uxlOIZ{>3xmkk zp>X@TwsXwu@occa%^&fF>kg%V2B?&u^);_J6$KQCjxME~d)S?R#iD#-6o)^$M$VmS zHVcS?p?Cn!<9@SdsFc1UBoZtdnRDo*WwHyLgTj-?C8hE%0Nsj3t47$e?tM7Wkj;4G z;%dUhK_V@fI0+o(3;?)e_9xmx$o~gmME_4CMKrhGW2!yoV>ssDxa>B-Yi9GhUKZdu5`f>^@&OcgPTFN2{by0P=drw6 zk3%c;PeTPq0BwPRmFZbbuugC2ZL-m zpxFS+0VaAqPt41ZNQ!gZYmUX>DO%94mYn?T0zWB=_aC5yrvRXzt>?ZoAXDH{VWu(Z z>6estYz2SiG4W&pY6N)VasGDDdEe%c=-YDDA<;;+7&NH^;xl<+^_4q#Ki)uh~d>pSZm3E%gTK z^Ga8ipWLK|+UUhQt*rKm7@w@rV0;qzMl#n!ru=5W6l9_|V|WNpXcQPIi1G%K@;A6k zD1MaP)jd<;W(dRqyqY92>wHa9xEE zLSzuLo2Xief|E9?hV42Sr#lQc0t#;KS}*gFW$aswWSP;%~o=#ioa970A_T ze-6aQls$0!!O#fE!RP0S4`9LfZ$*LmgigBJuciT6}(xT3V+l zRh+Jt%raf{j3SUf$fSDb1wv;7ljNCuNkrB#%+WT|rj{HDgs&Ed*<9}h1)NU5${-tnsux^R_ zJ~X!(?`32s^E0&X73>qNjWe5zFl&Odd0FBawB{{1x50*^zzJJi`LHc-iy1g|kXj}) zN5r9DCt-KoRysOl*V|2O-zV63S;^%FITRJ%DumA#o+&*6GxJx1++c%hT`(E5&7WzX z@c>bFkcER&ff`>I>hc&B)BieG6{liJ!h{U%DDzsj1phM+r)kB!ez#K}LHpQu>QFAQ z4?mW>lVtm^Vt$-#n&R(v0QR7JTuJ@TtWpM7YmhIoSz!HjDvjKPc}+!(Axj3&v? zKLt2=`d(S7?&_k2ecT4vW0o_#trioawMH8P7WD!uXTvtK@D}z25R;h?a&Q1_&`5>n z=dBl0t;G@rH|(h}XxW?xN!_F+bWYb^3BJhyO~H8L3$u|w*f#XL`KB4#n1Yx# z4WPEU=lhTaJ#Wn`lO!=dLp%LV&1D0=MX3`j8XD+w$}H%`GMc$J;*+adKJ`ZmOyzyK z@`ot&m!8LsO^uLe`LM#HwXINUWRrFQ86_H^h`LUOj!SINwR6Q`*!Bl(UP#jLQEi*_ z{1O6>R=mcAuETft$-Sb74=7S<<|y}LQ}j#rneqdj)bgW=CPE&gz_V05S8jA2^zJyg`H|6EH*K{5EgWrM4z(#c;{kyPv#|0x<8yKj?({)B^@Y|PH zY1{N4pS%!a6eZLn+K3~Y0Udp<)UZ(fr5*Is$XX-;VfunZk0bz~!(j8s992bRTr{s~ z=6v;N^g9#V8_zI*81PtbOZS61|1WmHLm@M&H*?)ntl^cOwc zVsCr&%Wh^9$;$F~MIZglT_PCDKdjQ*S;FzTz1j%)I~EPJNDp7U!5qB5GD~SuZ(>)V z#+GsDP|2 z&w@|6V7cFPprr>6&q+7=`K#(fG+PdcVX&KuX-+>$>jC3NumEN4aKn8GxQzUzpy^Nu z$4Ae-*Rw3YUdDchc)%)CPwBHDD@V~afoO#rcdgF~y>>(3y1l12+Oyrller9Wx;Vq{ zjw{fx47N$7g}7ei=W-|Z=8)e$h5XEds1Tn8A+}Mb(nH{??-bR6UB<6Zt;}ss!|`{zh9s=gK9Z_ zKbdDtJPBSE3n?~B1d{!4nmRo-Pwt!H>Kr_}%vq|q?S1AIG>kXk1oj0$g+e7Tc<5e+N{y!nz%6l&!C8p;{Y z<*sROZO{%BaDY&ZB+>KRz#SLkM#@yVea*(S(Z)p0uVC8vkgaCkIMbN?M9c`_MHXQI z`sD+P)0Yr{rz6CK;L;(~`#Ff}?hrl#N^oTrOj@*U0G8u&cd;MlFP*0c)Pa43I$(cG z1H=rGZuI1G-$mky1MdiBSxi1@+mvC(2HCd;t>M)(UOjW@BWxyVt1<tDe?L5jp>NGH8tnEn!MuX8dI9R) zI5GC@;lKHvgwK-o*{TLYa~-jRvdQUG`wleP7~Yt-4?N;@-y~k`f6+60A)eDcqC`@D9S5)SDAJTgzeWg4 zF}ONKq$c;(cII1Pizg(ul(OWo&Fim-`~dv8&NeV7K2AX)0d>0e2Xab)1&=PtrZQy( zc}z_lm6qAX9yA3R^-pCvs)E4FUe8}{7+YvV>vc4j(AkygIoi^lhFLj!OubENm$qN| z@4gF>A!H%BVgp_tOMm*o-ez}q6$_w{fM2dZU$(eLd+FGpnvs`mqwWHTg+RBR6_fg% zl`t=f3(A!92(j5c%fav!QBJ@uIp%1rCd(pldSI15^|}X~FpZ10G&V3`wt=pHq~i=l z-dSS60j?A&4)i@OK~iKz3A4Qiuwg<#G0jceJ5c+*=qFy9ivm$0(0v~(nk&j7insWJ zh&pfk4!d8rY^T}r*oD)-6`Ha>q<#|4ovXCR{@w$l(h&73y#T+9ite$-K9aj7tUUcQ zr2Z0!vSD`VQnt8Jdud!)7AWV+-D-RzfHn7G5iZ-Hm$7rAW|m>eK~lk%?fJM4NX#F5 zM}JO=Eztkb!`U_V(1Gpe{-yxdoyg(4D5k@GFJd;|ILNPie?;1cQB5w>4A_DgCg7Kg z7S$>#Q)kg3G&RIggyv_Ha&Kpb@XZ&jrR4gcJg2#=&xj7o!x7j)BSE9JM{MCTxKcL7^ zh_+;?hY@&jd6>o_#2$uZ@BylVy20u6iyBt&juS~oY?rk+O2Lg_K2cH!!PHn#OL(L& zUEeP!=&^BRzf?C`foJ0p@UG|j;F55bzG2(&U}Qyv{1hp7Z(7K`vfnEdM-Wfx<;Q_% zrQrJ@S?ujqaPoSS;`bPyRfrE>D4$hWnF}-Gw&RqtgnRt5iZOb70BM_ z#r`h)9Js)5i%~)6`7+k7*qXc34vvX}b+xF)gp(x)qGeyrgQf0#p>)ro%E%`cx?A;9T*tzLrK_*&T1S_|bRJ*r>;f^+BHTYvh4Yw`mUYBgxHsggV%Q#BUj;f zZ-(mx9>$oM=^r};)rR~0V2f_4-j1F&l~2EC$HgRz5nmRFq<}nHqvG8vZfZtq;*GdC zIQ5ddMVPJEYDo4bdypHCe8A0$%yXkSNIgZavF~jUj?M_kuUUHWFwI^+Q@d1^l#JFg zD}o?iJ-I5)ye}TFJ|?t`I^Ahsl7q*`f zG(&qtIib7|eKc*SH*T{h$DH_=iW=Ukccd$RIXb+i2Eap$pt+$2I+a~9(d926L(FXL zh^p_+OpaK#Cycd}=P0C0*J*3?>crWmyRM)0s>@-efoSnyDt&m-b_M;{S_r1lRPXlH z;n+-PI9nm_H-+%Tm)VogMrj%=Wu?|8=UQ~(-vkB@ErM?J>Q3Fn-uTf z#_-Z4D#u2jTP%iD5sw|E7hewKvVkYNZd7TEUASsM9+yS~!@~+^J3Vu%n~^kY_l`C~ zaO^W$uumiK|L*fGlB9LDWFNJB@sphrAv!c)dbpz|IkiOD4KdBc2m4e&lnbc(?8560 z3a!q(<|>8ftwvb(gg_7;D~<61c30H=cMK{gsLK$-3?OtC2!g3B$X0Rd`B(rPfxLkh ze*6cV90HML2!UvGXgw?FcYy;~-Ii|x64h=%#2N!<>B#r+Vpv=#^@{^g8?rOJlU^^U z0fZIt0Zy0;zH+@BrU&jd8=%EpW(BA8|I>4I9(*x&^`JE+2&HL1749etyzk8;11?-^ z>#(bgjH}dBS_e*K?iyexv;V_@Rg2zS#+QMPPTS;d8R~McfM(v~A2=bcoVk|Z*)uI+ zdmg|@FTMY--6?O-|BgB3B)jLrhZHEy6Zkv&*2BEl z<}yCH!B2;#f~AazH+DpIQ9$Y13J#!bLS+|9+Lc8=;yFk7Y=k?8F#Mz5El*{9zU+B= z)i|oV5@Guf)mZ#!T@rB-W$z}SF6D(KE(st_q(RX4S)@s+N4c7MBu1ng`Vv5#V0N;= zA5-FZjTfgD{Z)YWcpgh})dniX5_d6f=Hm?R^1r#dq)E$1xw)FUBt~SwwHoO>ayiOx znkW88s(9#Vj4CIZY@@=LynR#osO`?b4fV!ZI@}wtWk3)A!ov2wnGNtFW;G9`yy^O< zD1ZCbO9SZtYVOMep?d#+(I}Imj8Kv#1`)E1eVJ^LK?sEm!jLUXge-%Evdhlcx9qYL zLe}hJ>}B7xW{KZ3KA-!$_n&+3_mBI>?a$_%_j%59&hviW=lxpX(x%=2jU#YQem<|J z+iSdB_9$eRNbDjH=!g;C3{h$IRt=#_4@Uc%6iE*qN^lOyq7|Gbwe)pk(AUUrVr! z^!o52AA^+#1;ZluHeSekVLdzAeP3KKGUMDt)ga_}c{)HfW>#xcZWpwIf#9#w*RV+O zON5!lON~H1zrAMGf|vf7CJ4mzWTf3M=0<5^17rxSP&Nz&WYhNQKRn> zaOM3Q_Zl%4T^hmx79wgPOG2QEX7Xdtz*Jk><@E-4!wgH+R+Hk;M&e=Y@gp%VEjs%T zBNpX7IIc_bzo_AgRFTPYvVVCNpJ?&I2c63{)OgK^-kzQMTR@deywA)_72%s=y`4;F zjZD25k0rfHNbKoUDBtcT>N~)&Z*L8ME^Cn74GrD3F64^Yr<(c+6(UREllH{Gc z4_>MgE001PVx~z5C-|!~7vgvJ2n9GQAV{gQ%`s(tq84CX8aA0#_h zV++?y!py;e`Oju*C}yGpU{NHGNmJE}C`R514re5xGp$INZI8PX2MxhYL>}Z7#np7`*j(a3 zQeZ0b5Kzl^qCn<+Gep!v0&KN105RRQ%S6nO;eY;~K*)|2iqz!5e=C|0QCfnt#WP>P zkf9RlYN95)sMJI9BpsZ19zb#{T{;(Ru$h~0dn1xaOCEsumq>VMK4+Dj|B)g7n&_Y7 zyt!!j4JU#o&I9!YfwM*ji8a0_B}oX#hja#OoQZ=v2(~@s9S;CpS`Y$jUo3+7Uw+=C zC66uS5$7bWWaC4Aa~YTdw9xa7jmGypUi0>V0RYVT-m98HQohCyvDLfZ8|81_rFm5^ zmJ~@cP8EYQStWBKRg&wKBjMx?9X`x; zGbE~`cpTIs0MDpo0HajDdc~Hn@sT!MrAx#4c|{2n(y`4>u^-DcE=8K=#T_|*aB)#6 z`w=%EXG{KKDD))4Oa^xx@@0{B?9LlRqnHCSIi^KO1!twsnXTGw+!&#Ov$V5TXJ4eV z^qyE>=$T4YwY$XM_&m9bmer3d&MWEZ5?n<7ukk(Ep=mrA=k48w%D&TXqeFlQVals% zADImg-s1k+9D}irx4kcLS-be&m@!@r6)G*ID_?e-_Sh0@q4E8Y4|(}J7fp|6J)hyk zERyqhK_K0EA1oJu{`5T!P>-;qVd^v$(ew1B*E8TFc3C`gmP@LcpKP=nF{;-m zmA@7XBlrIJDJ>SEzh+&7g_1ReleN*(IcmoS1^THFA^~>cB3%xshG6fYi1Hqh&`(KU zRHwGKt*~nri_;R^Lby}N?!rX8HHtFMFj~j z<@_Sujv!}F#tUWns}enee-i*|rC+7OOnp~9uG&c=g&Qs6IA~%D3jq4RP6E5xVrCOR zWwHdZQ?lGh}ek^|lPc7Y3zniX`%7bZ&yr6 zsdpY<2D&ILa9{!1#*VssoW7&eRWB5^h7Zu^MRj}?hJ2Z zWVBgqY}bBE{to)be14BvMMNukIIyMOa2 z-NSoiv}2A5VBWT61m6zS2_bDZy}I7y<1E#~w1Y@Nu01*CfsfQJ`+&lTkWhk4248u0 z%;I687|W?mL0L7;BX^8;eIy0Io;o8?n06hCK?AtW8(*0i|J0^ zo%5)dI+0kxx-i&H!Sz zo3d%R-63ne9g@gxyrk8}zFPt%iP)1|z24L$|8M%QlrN5<`zkusO~nSE&Y0J|^xrNn zCAiyv6w)-ENT?JzSEMf+-PJtX9l% z2s^DMF(HccU+-K8#~%Zai$$WPwtB{dKwNw?XX@JDQqqKjTfx!=Z>+ z^CRGDVjx(xck%3KWO0o0uPZ4A&5@l0Y&pVk0!C5$$+JUyQ~!sK1w-l&N~2X4z=Os_ z(fc9{vcy?WHF?yc)8~)gQomMxUi^~bn`o_M>p`s=%Vu2(c!ES;+Eqx5Ih-W|zFYy{GL+jlgvU?ycH4pq*2pDcJ&kxEq6}^x(1~jaaEl0<%(MTd4$f<(2 zB2iZsDAUc~GXM8s`31&uR(s@O%i)8O2iwAL6o3ZzJr$EO9Oo)uuFQwpvmJB*#iUH0 zzGT^e5d`v>5~Bxf0;gq7v|&_V?`K6ki(yUtyaRY&OQ`?@4%VWFfMOc=xZoDPbM=Fa z{iqG9K}~&&l4T9qTTL8ryZh9?m5xiP01>tny$;F<0JWEO49SWQh_L#+tCDZqyR5!G zZ^*x)UiR{`;?Cc8UAjrzq_D&F( z_qfN^x9|?9XzUR{Mv2Olf5^uBf1Hr~AHM*Ob5uQDqR1!aopf{1%O|*|Ny=!toE0Vi zf0~4H-UF^%pOMCy)ub}<|5%qQcYWx~NAQ^y=n<6>>`kW&0z3Q3`dm{fT||2zdTurk zKT*1F#-dAt7v;y;CrEL?I78j1<_-^6eAWjgn!|S?dGXUqhNg%^BBE%}DZVit4_!1b zdZ0ozqUf&QD<;P)Ikfjn)cD%ntXZFC967Nduu!wK^9M|7npSSpubo3Eoca(tBc4bf zG-@sFqTx74ch-f;4#+p@YeZn%m$|&&5Ap7-ia-G_PJd=9THbA}Dh~akd~1=TZ=8W3 zA+K?wO_?!SQ&qOaqOZfph4Rhk({`;_uOgq{pX}rZ->o1a;?6sR><716=Og1(V(*ty z8y(#n!>8Sk&fA^l%y?yn8Y8abxTti=Pik@y;`%9o^{DV*u{lmY7*|oU5$hK$=BfrG zaylDMFOYw+DZM6qUs)=5Ufs}mWf;!D0r1f)tnkc3qIREG!|<7JG_A86$K&znZEE$F zhCRjJ7AeN^$nD=(cY;b(odYlFH2>M-5cDozVqIsEFutIk zTe0*6=G+J|f-mt-czI`81+xdjb=yS~Odz?8)bcnuUwarmxHN!qEbVOJxUz@R8LBb| z!WaE~1$H~iuQKp^-%PWrbo1_8mhJdQ^89V@=8NiSLrX@Yx||AWCf(j&bWhLU09+)Y zN@-ZSL1|>UX10B(na3As$&X$4snBpnR6-njruzMwX!^GI5$hibt%FT-cMW6?{dF$^ z@H{TB_d$>>IWBRw9ZwhwVb4+M_)|#{Sg&o>*THg%d)pr-AIK@*yfL!p=-5C{^!aSo z`yl$9MM|RaE=%qD(%vg-+OX$GP;LMD{8zxS)xgM>4>!jq>2(Z-`%l!#e!mn~J~43u%N4DZeJBE}Jd`WgLL0Iw{&@@A?87jSd;p~V z8{-*J4$W;L@wv5kwmZqTvlpd4aw=!|B|CFYC`R-p96>~C98T@UlDux|hta}=IBWsM z*7bJB5^=^WMB|;=M$)}|FR)DQlTxfclnR=XH@p0dY51N8RE?uU4R_>tOJ9e>O%_@YcdzC(3f1?B{@R*v2=WM$3euf91 z!_9J8JOQ-f`*mL(N|aPeVtXS>#7hRQJR+4TIl?HbWoce+ua)hCOE<^BYOSZBf11a# za%PXvEf-eq3s$?by_WBh>*-{bZ`W27Ch7$#_Wy42_#e1H{^y6JPxL0;dYYd@(f6BF z)Umv^Rn%z`9T5}5{(d(0XIeYP?fXTImp`B6*MeR>@XLKA>$Yy@^2IBwiGNz2e|s0v zk+a>Auswqj3_JL#lg3wSZ*4k!L7Qvi4#>)uvH~S2=a9UaRid#qtC* z3l#XT9J5*fE=q;c%sj2gg;l>;%kcaF_iUZSQ`eMv>em`*JsrAzq?pLp2rGQXpCUV4 zTQ==(KL|Or=rG)E&VM^bV4$6E-{3WUp$Ly^DXZz(a+thERpX^leNmG~G@1f@`B3HC zJT^<_DqVSFzG-e{aAP?CG}0s0WcD;02G@;kv+-@kuf;%j23+cw7ZVT}V`tk&GbCXv z^P%!D3(b~sxiB*kY=I~2**mA|R4$M9kQW`MjT|ocb=skA25rGcd`6ECw?O9(^e8&P zg5*TVaDZK`s#WtOM9he!;-V;nd8w-q@F-cHYqw3;Je$5jh@(>NL}4i)6#m7(K^hz9 zGpblbc&^xch)VchL)pb?REP9K+`OTSu)bZ7lnWyT*J7AY6+fF}S9K)2wd)9CuLh>} z7&LYBx;CToazwyAJnsIIAS&>on{}P-84lqd|sb>cXw_UkBDR&6a8^ftholmO-M9IGd;6812nQo?G9 zw&}0E?O&I*cr5TdaDz`wc+MCeo+cD~Vb+^Ng}#%M>!R8%*r$F(?jS@h?yZo}fO$D3 z>*#PYAlGvqsQwCvZh30CK0E3YaPmrn$Mh#(d%NSlr}4!KyOZ~fR$3lt8h~4`D6H4C zE}hNo3k4fjyV>ZubiF8#Eo(%-c}RwC$QeiKk%fM8 zPt%&IA6f8#fJIY@q=(;?Y{PmN^!vqs7ib`mlUj1^kOS$CBw?JiTy3N<(q<}28+GM3 zhd{aVt1jJ9$a>y)kEUBd*Ua}|3*P%aLwFNI^F7O@TYr2ZwvhG@McTJjQ%_8Mi#2~% zf>p?P+mRKDJL}1&Y9GX+36-3B^Tkw@6tV5<3jHXL(HjFqPm&<=BqViPnI~k`D7K52 zQK~dq$UE=Bqs%ECtMSI9!Z;_e{Q=EZ1Jmy~rt&Cnb~nSIACMhBgGKX=erYsG&BLsp z?AH8h8_Fe#M5C=tlExu5dda<6RQqj%+MMHK8rR?vOS`dV>_Ht01Lj}NqT z5y%wPP|@)ufr*|s{1fdlGI?xn9r>~}f78Zic2#hmv=JTxOO11dPj7WPaySwiw#aY+ z?^iR`ZVq%du04|h$&=g0eF~0^4s?lQ>o0zIq2^jN(;bOQ5WHQsmVa@e>9Cle4G zPrDFZZ)toAJt9~&Ftx{d6~rBg2{FjB-96li2rjBvXuD>G7?^wNE8*WKi};J5A&u)= zdK{a8ik?Prem`J~{eA9!>y4*TIL%S#QG6EvK80!AlR!9G>bP}c7c*>;@i8Cq-|&hQ z-lZ7m$Jd&7*kFq~sDz}zoQMA=tpUH_ozxdAj_T9~9ou{On3yc5Bn^^?vYPigJ3#4H z=?3@fH%d5E5s7}Ls}$|}gt4C|W3HXW_;wMZ|2L$pln%Au9=Ug*a#b>0vWfdjzt~~2zq3@Qy>juh3-*$;lwVai>HB(#&DfYKCpMH@! zJeMja3B>m{Eot~9M(i?;-Xx`ySW0A*QRHEH8mXjQ z^bQt?!bA=A6zQ#*U#G6P32TOYB8_dwmovKD57}y#YjT%0elX=LW%SDOHrp2sR2io> znFRF-M}_NL|7W%QMS}{5j^(Y^r3PYQzXZKHjd=$C8L^eNV`ZTw9aOKZpF$pB?t?T^ zK#$LX<_;9T+QCyF$rBVWz*^b{v#`OKFW18pw}KVwBqFL<$W4*lowD{!NSf<#RdJmc zCA{tSExq%TV?PrUuIcXN{Y}5M@(h?$gP7_Cbod~59XnH&x&ZTctmfIjKc21Y|HN|o zKQA`^PsY{%+bABy**+iXi5fzC|D4C&cxnUw{KWs0e?De{ho{6jPR* + + 4.0.0 + + + com.intel.oap + remote-shuffle-parent + 1.1.0 + + shuffle-daos + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-source + validate + + add-source + + + + src/main/scala + + + + + add-test-source + validate + + add-test-source + + + + src/test/scala + + + + + + + net.alchim31.maven + scala-maven-plugin + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile-first + process-test-resources + + testCompile + + + + attach-scaladocs + verify + + doc-jar + + + + + ${scala.version} + true + true + incremental + + -unchecked + -deprecation + -feature + -explaintypes + -target:jvm-1.8 + + + -Xms1024m + -Xmx1024m + -XX:ReservedCodeCacheSize=1g + + + -source + ${java.version} + -target + ${java.version} + -Xlint:all,-serial,-path,-try + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + compile + + compile + + + + + ${java.version} + ${java.version} + + + + org.apache.maven.plugins + maven-surefire-plugin + + + org.scalatest + scalatest-maven-plugin + + + test + + test + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + false + true + + src/main/java + src/main/scala + + + src/test/java + src/test/scala + + dev/checkstyle.xml + dev/checkstyle.license + ${basedir}/target/checkstyle-output.xml + ${java.version} + ${java.version} + + + + compile + + check + + + + + + org.scalastyle + scalastyle-maven-plugin + + false + true + false + true + + src/main/scala + + + src/test/scala + + scalastyle-config.xml + target/scalastyle-output.xml + UTF-8 + UTF-8 + + + + compile + + check + + + + + + + + + + org.scala-lang + scala-library + + + org.apache.spark + spark-core_2.12 + + + io.daos + daos-java + 1.1.0-SNAPSHOT + + + junit + junit + test + + + org.scalatest + scalatest_2.12 + test + + + org.apache.spark + spark-core_2.12 + tests + test-jar + test + + + org.mockito + mockito-core + test + + + org.powermock + powermock-module-junit4 + test + + + org.powermock + powermock-api-mockito2 + test + + + + \ No newline at end of file diff --git a/shuffle-daos/scalastyle-config.xml b/shuffle-daos/scalastyle-config.xml new file mode 100644 index 00000000..4d097a0c --- /dev/null +++ b/shuffle-daos/scalastyle-config.xml @@ -0,0 +1,392 @@ + + + + + Scalastyle standard configuration + + + + + + + + + + + + + + + + + + + + + + + + true + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ARROW, EQUALS, ELSE, TRY, CATCH, FINALLY, LARROW, RARROW + + + + + + ARROW, EQUALS, COMMA, COLON, IF, ELSE, DO, WHILE, FOR, MATCH, TRY, CATCH, FINALLY, LARROW, RARROW + + + + + + + + + ^FunSuite[A-Za-z]*$ + Tests must extend org.apache.spark.SparkFunSuite instead. + + + + + ^println$ + + + + + spark(.sqlContext)?.sparkContext.hadoopConfiguration + + + + + @VisibleForTesting + + + + + Runtime\.getRuntime\.addShutdownHook + + + + + mutable\.SynchronizedBuffer + + + + + Class\.forName + + + + + Await\.result + + + + + Await\.ready + + + + + (\.toUpperCase|\.toLowerCase)(?!(\(|\(Locale.ROOT\))) + + + + + throw new \w+Error\( + + + + + + JavaConversions + Instead of importing implicits in scala.collection.JavaConversions._, import + scala.collection.JavaConverters._ and use .asScala / .asJava methods + + + + org\.apache\.commons\.lang\. + Use Commons Lang 3 classes (package org.apache.commons.lang3.*) instead + of Commons Lang 2 (package org.apache.commons.lang.*) + + + + extractOpt + Use jsonOption(x).map(.extract[T]) instead of .extractOpt[T], as the latter + is slower. + + + + + java,scala,3rdParty,spark + javax?\..* + scala\..* + (?!org\.apache\.spark\.).* + org\.apache\.spark\..* + + + + + + COMMA + + + + + + \)\{ + + + + + (?m)^(\s*)/[*][*].*$(\r|)\n^\1 [*] + Use Javadoc style indentation for multiline comments + + + + case[^\n>]*=>\s*\{ + Omit braces in case clauses. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 800> + + + + + 30 + + + + + 10 + + + + + 50 + + + + + + + + + + + -1,0,1,2,3 + + + diff --git a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/BoundThreadExecutors.java b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/BoundThreadExecutors.java new file mode 100644 index 00000000..c245f3b0 --- /dev/null +++ b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/BoundThreadExecutors.java @@ -0,0 +1,204 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + + +package org.apache.spark.shuffle.daos; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Pool of simple shared executors. + * User should call {@link #nextExecutor()} to get next sharable executor which is a instance + * of {@link SingleThreadExecutor}. + * + * Before JVM exiting, {@link #stop()} should be called to release all executors. + */ +public class BoundThreadExecutors { + + private final int threads; + + private final String name; + + private final SingleThreadExecutor[] executors; + + private AtomicInteger idx = new AtomicInteger(0); + + private ThreadFactory threadFactory; + + private volatile boolean stopped; + + public static final int NOT_STARTED = 0; + public static final int STARTING = 1; + public static final int STARTED = 2; + public static final int STOPPING = 3; + public static final int STOPPED = 4; + + private static final Logger logger = LoggerFactory.getLogger(BoundThreadExecutors.class); + + public BoundThreadExecutors(String name, int threads, ThreadFactory threadFactory) { + this.name = name; + this.threads = threads; + this.threadFactory = threadFactory; + this.executors = new SingleThreadExecutor[threads]; + } + + /** + * get next executor in a round-robin way. + * User should submit all its tasks to the returned executor instead of getting more executor. + * + * @return instance of {@link SingleThreadExecutor} + */ + public SingleThreadExecutor nextExecutor() { + int i = idx.getAndIncrement()%threads; + SingleThreadExecutor executor = executors[i]; + if (executor == null) { + synchronized (this) { + executor = new SingleThreadExecutor(threadFactory); + executors[i] = executor; + } + } + executor.startThread(); + return executor; + } + + /** + * interrupt all running tasks and stop all executors. + */ + public void stop() { + if (stopped) { + return; + } + for (SingleThreadExecutor executor : executors) { + if (executor != null) { + executor.interrupt(); + } + } + boolean allStopped; + boolean timeout = false; + int count = 0; + while (true) { + allStopped = true; + for (SingleThreadExecutor executor : executors) { + if (executor != null) { + allStopped &= (executor.state.get() == STOPPED); + } + } + if (allStopped) { + break; + } + if (count >= 5) { + timeout = true; + break; + } + try { + Thread.sleep(200); + } catch (InterruptedException e) { + logger.warn("interrupted when waiting for all executors stopping, " + name, e); + timeout = true; + break; + } + count++; + } + for (int i = 0; i < executors.length; i++) { + executors[i] = null; + } + stopped = true; + logger.info("BoundThreadExecutors stopped" + (timeout ? " with some threads still running." : ".")); + } + + /** + * An executor backed by single thread + */ + public static class SingleThreadExecutor implements Executor { + private Thread thread; + private String name; + private AtomicInteger state = new AtomicInteger(0); + private BlockingQueue queue = new LinkedBlockingQueue<>(); + // TODO: handle task failure and restart of thread + private Runnable parentTask = () -> { + Runnable runnable; + try { + while (!Thread.currentThread().isInterrupted()) { + runnable = queue.take(); + try { + runnable.run(); + } catch (Exception e) { + logger.info("failed to run " + runnable, e); + } + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + if (logger.isDebugEnabled()) { + logger.debug("thread interrupted. " + getName()); + } + } finally { + state.set(STOPPED); + } + }; + + public SingleThreadExecutor(ThreadFactory threadFactory) { + this.thread = threadFactory.newThread(parentTask); + name = thread.getName(); + } + + private String getName() { + return name; + } + + public void interrupt() { + thread.interrupt(); + state.set(STOPPING); + thread = null; + queue.clear(); + queue = null; + } + + @Override + public void execute(Runnable runnable) { + try { + queue.put(runnable); + } catch (InterruptedException e) { + throw new RuntimeException("cannot add task to thread " + thread.getName(), e); + } + } + + private void startThread() { + if (state.get() == NOT_STARTED) { + if (state.compareAndSet(NOT_STARTED, STARTING)) { + try { + thread.start(); + } finally { + if (!state.compareAndSet(STARTING, STARTED)) { + throw new IllegalStateException("failed to start thread " + thread.getName()); + } + } + } + } + } + } + +} diff --git a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosReader.java b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosReader.java new file mode 100644 index 00000000..fde2ac56 --- /dev/null +++ b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosReader.java @@ -0,0 +1,244 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos; + +import io.daos.obj.DaosObject; +import io.daos.obj.IODataDesc; +import io.netty.util.internal.ObjectPool; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Tuple2; + +import java.util.Map; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; + +/** + * A class with {@link DaosObject} wrapped to read data from DAOS in either caller's thread or + * dedicated executor thread. The actual read is performed by {@link DaosObject#fetch(IODataDesc)}. + */ +public class DaosReader { + + private DaosObject object; + + private Map bufferSourceMap = new ConcurrentHashMap<>(); + + private BoundThreadExecutors executors; + + private Map readerMap; + + private static Logger logger = LoggerFactory.getLogger(DaosReader.class); + + /** + * construct DaosReader with object and dedicated read executors. + * + * @param object + * opened DaosObject + * @param executors + * null means read in caller's thread. Submit {@link ReadTask} to dedicate executor retrieved by + * {@link #nextReaderExecutor()} otherwise. + */ + public DaosReader(DaosObject object, BoundThreadExecutors executors) { + this.object = object; + this.executors = executors; + } + + public DaosObject getObject() { + return object; + } + + public boolean hasExecutors() { + return executors != null; + } + + /** + * next executor. null if there is no executors being set. + * + * @return shareable executor instance. null means no executor set. + */ + public BoundThreadExecutors.SingleThreadExecutor nextReaderExecutor() { + if (executors != null) { + return executors.nextExecutor(); + } + return null; + } + + /** + * release resources of all {@link org.apache.spark.shuffle.daos.DaosShuffleInputStream.BufferSource} + * bound with this reader. + */ + public void close() { + // force releasing + bufferSourceMap.forEach((k, v) -> k.cleanup(true)); + bufferSourceMap.clear(); + if (readerMap != null) { + readerMap.remove(this); + readerMap = null; + } + } + + @Override + public String toString() { + return "DaosReader{" + + "object=" + object + + '}'; + } + + /** + * register buffer source for resource cleanup. + * + * @param source + * BufferSource instance + */ + public void register(DaosShuffleInputStream.BufferSource source) { + bufferSourceMap.put(source, 1); + } + + /** + * unregister buffer source if source is release already. + * + * @param source + * BufferSource instance + */ + public void unregister(DaosShuffleInputStream.BufferSource source) { + bufferSourceMap.remove(source); + } + + /** + * set global readMap and hook this reader for releasing resources. + * + * @param readerMap + * global reader map + */ + public void setReaderMap(Map readerMap) { + readerMap.put(this, 0); + this.readerMap = readerMap; + } + + /** + * Task to read from DAOS. Task itself is cached to reduce GC time. + * To reuse task for different reads, prepare and reset {@link ReadTaskContext} by calling + * {@link #newInstance(ReadTaskContext)} + */ + static final class ReadTask implements Runnable { + private ReadTaskContext context; + private final ObjectPool.Handle handle; + + private static final ObjectPool objectPool = ObjectPool.newPool(handle -> new ReadTask(handle)); + + private static final Logger log = LoggerFactory.getLogger(ReadTask.class); + + static ReadTask newInstance(ReadTaskContext context) { + ReadTask task = objectPool.get(); + task.context = context; + return task; + } + + private ReadTask(ObjectPool.Handle handle) { + this.handle = handle; + } + + @Override + public void run() { + boolean cancelled = context.cancelled; + try { + if (!cancelled) { + context.object.fetch(context.desc); + } + } catch (Exception e) { + log.error("failed to read for " + context.desc, e); + } finally { + // release desc buffer and keep data buffer + context.desc.release(cancelled); + context.signal(); + context = null; + handle.recycle(this); + } + } + } + + /** + * Context for read task. It holds all other object to read and sync between caller thread and read thread. + * It should be cached in caller thread for reusing. + */ + static final class ReadTaskContext extends LinkedTaskContext { + + /** + * constructor with all parameters. Some of them can be reused later. + * + * @param object + * DAOS object to fetch data from DAOS + * @param counter + * counter to indicate how many data ready for being consumed + * @param takeLock + * lock to work with notEmpty condition to signal caller thread there is data ready to be consumed + * @param notEmpty + * condition to signal there is some data ready + * @param desc + * desc object to describe which part of data to be fetch and hold returned data + * @param mapReduceId + * to track which map reduce ID this task fetches data for + */ + ReadTaskContext(DaosObject object, AtomicInteger counter, Lock takeLock, Condition notEmpty, + IODataDesc desc, Object mapReduceId) { + super(object, counter, takeLock, notEmpty); + this.desc = desc; + this.morePara = mapReduceId; + } + + @Override + public ReadTaskContext getNext() { + return (ReadTaskContext) next; + } + + public Tuple2 getMapReduceId() { + return (Tuple2) morePara; + } + } + + /** + * Thread factory for DAOS read tasks. + */ + protected static class ReadThreadFactory implements ThreadFactory { + private AtomicInteger id = new AtomicInteger(0); + + @Override + public Thread newThread(Runnable runnable) { + Thread t; + String name = "daos_read_" + id.getAndIncrement(); + if (runnable == null) { + t = new Thread(name); + } else { + t = new Thread(runnable, name); + } + t.setDaemon(true); + t.setUncaughtExceptionHandler((thread, throwable) -> + logger.error("exception occurred in thread " + name, throwable)); + return t; + } + } + +} diff --git a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleIO.java b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleIO.java new file mode 100644 index 00000000..26990085 --- /dev/null +++ b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleIO.java @@ -0,0 +1,284 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos; + +import io.daos.obj.DaosObjClient; +import io.daos.obj.DaosObject; +import io.daos.obj.DaosObjectException; +import io.daos.obj.DaosObjectId; +import org.apache.spark.SparkConf; +import org.apache.spark.launcher.SparkLauncher; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * IO object for read and write to DAOS. It initializes and creates following resources. + * - DAOS client, {@link DaosObjClient} + * - read/write executors (if fromOtherThreads configured true), {@link BoundThreadExecutors} + * - {@link DaosReader} objects + * - {@link DaosWriter} objects + * + * It does the cleanup of all above resources in {@link #close()}. + * + */ +public class DaosShuffleIO { + + private DaosObjClient objClient; + + private SparkConf conf; + + private Map driverConf; + + private String poolId; + + private String contId; + + private String ranks; + + private boolean removeShuffleData; + + private DaosWriter.WriteConfig writeConfig; + + private Map readerMap = new ConcurrentHashMap<>(); + + private Map writerMap = new ConcurrentHashMap<>(); + + private Map objectMap = new ConcurrentHashMap<>(); + + private BoundThreadExecutors readerExes; + + private BoundThreadExecutors writerExes; + + private static final Logger logger = LoggerFactory.getLogger(DaosShuffleIO.class); + + /** + * constructor with {@link SparkConf}. + * reader and writer are created here if fromOtherThread is true. + * + * @param conf + */ + public DaosShuffleIO(SparkConf conf) { + this.conf = conf; + this.writeConfig = loadWriteConfig(conf); + this.readerExes = createReaderExes(); + this.writerExes = createWriterExes(); + this.removeShuffleData = (boolean)conf.get(package$.MODULE$.SHUFFLE_DAOS_REMOVE_SHUFFLE_DATA()); + } + + protected static DaosWriter.WriteConfig loadWriteConfig(SparkConf conf) { + DaosWriter.WriteConfig config = new DaosWriter.WriteConfig(); + config.warnSmallWrite((boolean)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_WARN_SMALL_SIZE())); + config.bufferSize((int) ((long)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_SINGLE_BUFFER_SIZE()) + * 1024 * 1024)); + config.minSize((int) ((long)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_MINIMUM_SIZE()) * 1024)); + config.timeoutTimes((int)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_WAIT_DATA_TIMEOUT_TIMES())); + config.waitTimeMs((int)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_WAIT_MS())); + config.totalInMemSize((long)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_MAX_BYTES_IN_FLIGHT()) * 1024); + config.totalSubmittedLimit((int)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_SUBMITTED_LIMIT())); + config.threads((int)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_THREADS())); + config.fromOtherThreads((boolean)conf + .get(package$.MODULE$.SHUFFLE_DAOS_WRITE_IN_OTHER_THREAD())); + logger.info("write configs, " + config); + return config; + } + + private BoundThreadExecutors createWriterExes() { + if (writeConfig.isFromOtherThreads()) { + BoundThreadExecutors executors; + int threads = writeConfig.getThreads(); + if (threads == -1) { + threads = conf.getInt(SparkLauncher.EXECUTOR_CORES, 1); + } + executors = new BoundThreadExecutors("write_executors", threads, + new DaosWriter.WriteThreadFactory()); + logger.info("created BoundThreadExecutors with " + threads + " threads for write"); + return executors; + } + return null; + } + + private BoundThreadExecutors createReaderExes() { + boolean fromOtherThread = (boolean)conf + .get(package$.MODULE$.SHUFFLE_DAOS_READ_FROM_OTHER_THREAD()); + if (fromOtherThread) { + BoundThreadExecutors executors; + int threads = (int)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_THREADS()); + if (threads == -1) { + threads = conf.getInt(SparkLauncher.EXECUTOR_CORES, 1); + } + executors = new BoundThreadExecutors("read_executors", threads, + new DaosReader.ReadThreadFactory()); + logger.info("created BoundThreadExecutors with " + threads + " threads for read"); + return executors; + } + return null; + } + + /** + * connect DAOS server. + * + * @param driverConf + * @throws IOException + */ + public void initialize(Map driverConf) throws IOException { + this.driverConf = driverConf; + poolId = conf.get(package$.MODULE$.SHUFFLE_DAOS_POOL_UUID()); + contId = conf.get(package$.MODULE$.SHUFFLE_DAOS_CONTAINER_UUID()); + ranks = conf.get(package$.MODULE$.SHUFFLE_DAOS_POOL_RANKS()); + if (poolId == null || contId == null) { + throw new IllegalArgumentException("DaosShuffleManager needs pool id and container id"); + } + + objClient = new DaosObjClient.DaosObjClientBuilder() + .poolId(poolId).containerId(contId).ranks(ranks) + .build(); + } + + private long parseAppId(String appId) { + return Long.valueOf(appId.replaceAll("\\D", "")); + } + + /** + * get DaosWriter with shuffle object created and opened. + * + * Should be called in the Driver after TaskScheduler registration and + * from the start in the Executor. + * + * @param numPartitions + * @param shuffleId + * @param mapId + * @return DaosWriter object for specific shuffle and map + * @throws {@link IOException} + */ + public DaosWriter getDaosWriter(int numPartitions, int shuffleId, long mapId) + throws IOException { + long appId = parseAppId(conf.getAppId()); + if (logger.isDebugEnabled()) { + logger.debug("getting daoswriter for app id: " + appId + ", shuffle id: " + shuffleId + ", map id: " + mapId + + ", numPartitions: " + numPartitions); + } + DaosWriter.WriteParam param = new DaosWriter.WriteParam(); + param.numPartitions(numPartitions) + .shuffleId(shuffleId) + .mapId(mapId) + .config(writeConfig); + DaosWriter writer = new DaosWriter(param, getObject(appId, shuffleId), + writerExes == null ? null : writerExes.nextExecutor()); + writer.setWriterMap(writerMap); + return writer; + } + + /** + * get DaosReader with shuffle object created and opened. + * + * @param shuffleId + * @return DaosReader + * @throws DaosObjectException + */ + public DaosReader getDaosReader(int shuffleId) throws DaosObjectException { + long appId = parseAppId(conf.getAppId()); + if (logger.isDebugEnabled()) { + logger.debug("getting daosreader for app id: " + appId + ", shuffle id: " + shuffleId); + } + DaosReader reader = new DaosReader(getObject(appId, shuffleId), readerExes); + reader.setReaderMap(readerMap); + return reader; + } + + private String getKey(long appId, int shuffleId) { + return appId + "" + shuffleId; + } + + private DaosObject getObject(long appId, int shuffleId) throws DaosObjectException { + String key = getKey(appId, shuffleId); + DaosObject object = objectMap.get(key); + if (object == null) { + DaosObjectId id = new DaosObjectId(appId, shuffleId); + id.encode(); + object = objClient.getObject(id); + objectMap.putIfAbsent(key, object); + DaosObject activeObject = objectMap.get(key); + if (activeObject != object) { // release just created DaosObject + object.close(); + object = activeObject; + } + } + // open just once in multiple threads + if (!object.isOpen()) { + synchronized (object) { + object.open(); + } + } + return object; + } + + /** + * remove shuffle object. + * + * @param shuffleId + * @return + */ + public boolean removeShuffle(int shuffleId) { + long appId = parseAppId(conf.getAppId()); + logger.info("punching daos object for app id: " + appId + ", shuffle id: " + shuffleId); + try { + DaosObject object = objectMap.remove(getKey(appId, shuffleId)); + if (object != null) { + if (removeShuffleData) { + object.punch(); + } + object.close(); + } + } catch (Exception e) { + logger.error("failed to punch object", e); + return false; + } + return true; + } + + /** + * releasing all resources, including disconnect from DAOS server. + * + * @throws IOException + */ + public void close() throws IOException { + if (readerExes != null) { + readerExes.stop(); + readerMap.keySet().forEach(r -> r.close()); + readerMap.clear(); + readerExes = null; + } + if (writerExes != null) { + writerExes.stop(); + writerMap.keySet().forEach(r -> r.close()); + writerMap.clear(); + writerExes = null; + } + objClient.forceClose(); + } +} diff --git a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleInputStream.java b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleInputStream.java new file mode 100644 index 00000000..264168a3 --- /dev/null +++ b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleInputStream.java @@ -0,0 +1,690 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos; + +import io.daos.obj.DaosObject; +import io.daos.obj.IODataDesc; +import io.netty.buffer.ByteBuf; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkEnv; +import org.apache.spark.shuffle.ShuffleReadMetricsReporter; +import org.apache.spark.storage.BlockId; +import org.apache.spark.storage.BlockManagerId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Tuple2; +import scala.Tuple3; + +import javax.annotation.concurrent.NotThreadSafe; +import java.io.IOException; +import java.io.InputStream; +import java.util.*; +import java.util.concurrent.*; + +@NotThreadSafe +/** + * A inputstream for reading shuffled data being consisted of multiple map outputs. + * + * All records in one specific map output are from same KryoSerializer or Java serializer. To facilitate reading + * multiple map outputs in this one inputstream, the read methods return -1 to indicate the completion of current + * map output. Caller should call {@link DaosShuffleInputStream#isCompleted()} to check if all map outputs are read. + * + * To read more data from next map output, user should call {@link #nextMap()} before read. + */ +public class DaosShuffleInputStream extends InputStream { + + private DaosReader reader; + + private DaosObject object; + + private BoundThreadExecutors.SingleThreadExecutor executor; + + private ReaderConfig config; + + private ShuffleReadMetricsReporter metrics; + + private boolean fromOtherThread; + + private volatile boolean cleaned; + + private boolean completed; + + // ensure the order of partition + // (mapid, reduceid) -> (length, BlockId, BlockManagerId) + private LinkedHashMap, Tuple3> partSizeMap; + private Iterator> mapIdIt; + + private BufferSource source; + + private static final Logger log = LoggerFactory.getLogger(DaosShuffleInputStream.class); + + /** + * constructor with ordered map outputs info. Check {@link ReaderConfig} for more paras controlling + * how data being read from DAOS. + * + * @param reader + * daos reader + * @param partSizeMap + * ordered map outputs info. They are organize as (mapid, reduceid) -> (length, BlockId, BlockManagerId) + * @param maxBytesInFlight + * how many bytes can be read concurrently + * @param maxMem + * maximum data can be put in memory + * @param metrics + * read metrics + */ + public DaosShuffleInputStream( + DaosReader reader, + LinkedHashMap, Tuple3> partSizeMap, + long maxBytesInFlight, long maxMem, ShuffleReadMetricsReporter metrics) { + this.partSizeMap = partSizeMap; + this.reader = reader; + this.config = new ReaderConfig(maxBytesInFlight, maxMem); + this.fromOtherThread = config.fromOtherThread; + if (fromOtherThread) { + this.executor = reader.nextReaderExecutor(); + } + this.source = new BufferSource(executor); + reader.register(source); + this.object = reader.getObject(); + this.metrics = metrics; + this.mapIdIt = partSizeMap.keySet().iterator(); + } + + public BlockId getCurBlockId() { + if (source.lastMapReduceIdForSubmit == null) { + return null; + } + return partSizeMap.get(source.lastMapReduceIdForSubmit)._2(); + } + + public BlockManagerId getCurOriginAddress() { + if (source.lastMapReduceIdForSubmit == null) { + return null; + } + return partSizeMap.get(source.lastMapReduceIdForSubmit)._3(); + } + + public long getCurMapIndex() { + if (source.lastMapReduceIdForSubmit == null) { + return -1; + } + return source.lastMapReduceIdForSubmit._1; + } + + @Override + public int read() throws IOException { + while (!completed) { + ByteBuf buf = source.nextBuf(); + if (buf == null) { // reach end + complete(); + return -1; + } + if (source.newMap) { // indication to close upper layer object inputstream + return -1; + } + if (buf.readableBytes() >= 1) { + return buf.readByte(); + } + } + return -1; + } + + @Override + public int read(byte[] bytes) throws IOException { + return read(bytes, 0, bytes.length); + } + + @Override + public int read(byte[] bytes, int offset, int length) throws IOException { + int len = length; + while (!completed) { + ByteBuf buf = source.nextBuf(); + if (buf == null) { // reach end + complete(); + int r = length - len; + return r == 0 ? -1 : r; + } + if (source.newMap) { // indication to close upper layer object inputstream + int r = length - len; + return r == 0 ? -1 : r; + } + if (len <= buf.readableBytes()) { + buf.readBytes(bytes, offset, len); + return length; + } + int maxRead = buf.readableBytes(); + buf.readBytes(bytes, offset, maxRead); + offset += maxRead; + len -= maxRead; + } + return -1; + } + + /** + * upper layer should call this method to read more map output + */ + public void nextMap() { + source.newMap = false; + } + + private void complete() throws IOException { + if (!completed) { + source.checkPartitionSize(); + source.checkTotalPartitions(); + completed = true; + } + } + + private void cleanup() { + if (!cleaned) { + boolean allReleased = source.cleanup(false); + if (allReleased) { + reader.unregister(source); + } + source = null; + cleaned = true; + completed = true; + } + } + + /** + * no actual close if it's not completed yet. + */ + @Override + public void close() { + close(false); + } + + /** + * force close stream when task is competed or error is occurred. + * + * @param force + */ + public void close(boolean force) { + if (force || completed) { + cleanup(); + } + } + + public boolean isCompleted() { + return completed; + } + + /** + * Source of map output data. User just calls {@link #nextBuf()} and reads from buffer repeatedly until no buffer + * returned. + * BufferSource does all other dirty things, like when and how (caller thread or from dedicated thread) to + * read from DAOS as well as controlling buffer size and task batch size. + * It also has some fault tolerance ability, like re-read from caller thread if task doesn't respond from the + * dedicated threads. + */ + public class BufferSource extends TaskSubmitter { + private DaosReader.ReadTaskContext selfCurrentCtx; + private IODataDesc currentDesc; + private IODataDesc.Entry currentEntry; + private long currentPartSize; + + private int entryIdx; + private Tuple2 curMapReduceId; + private Tuple2 lastMapReduceIdForSubmit; + private Tuple2 lastMapReduceIdForReturn; + private int curOffset; + private boolean newMap; + + private int totalParts = partSizeMap.size(); + private int partsRead; + + protected BufferSource(BoundThreadExecutors.SingleThreadExecutor executor) { + super(executor); + } + + /** + * invoke this method when fromOtherThread is false. + * + * @return + * @throws {@link IOException} + */ + public ByteBuf readBySelf() throws IOException { + if (lastCtx != null) { // duplicated IODataDescs which were submitted to other thread, but cancelled + ByteBuf buf = readDuplicated(false); + if (buf != null) { + return buf; + } + } + // all submitted were duplicated. Now start from mapId iterator. + IODataDesc desc = createNextDesc(config.maxBytesInFlight); + return getBySelf(desc, lastMapReduceIdForSubmit); + } + + /** + * get available buffer after iterating current buffer, next buffer in current desc and next desc. + * + * @return buffer with data read from DAOS + * @throws IOException + */ + public ByteBuf nextBuf() throws IOException { + ByteBuf buf = tryCurrentEntry(); + if (buf != null) { + return buf; + } + // next entry + buf = tryCurrentDesc(); + if (buf != null) { + return buf; + } + // from next partition + if (fromOtherThread) { + // next ready queue + if (headCtx != null) { + return tryNextTaskContext(); + } + // get data by self and submit request for remaining data + return getBySelfAndSubmitMore(config.minReadSize); + } + // get data by self after fromOtherThread disabled + return readBySelf(); + } + + private ByteBuf tryNextTaskContext() throws IOException { + // make sure there are still some read tasks waiting/running/returned from other threads + // or they are readDuplicated by self + if (totalSubmitted == 0 || selfCurrentCtx == lastCtx) { + return getBySelfAndSubmitMore(config.maxBytesInFlight); + } + if (totalSubmitted < 0) { + throw new IllegalStateException("total submitted should be no less than 0. " + totalSubmitted); + } + try { + IODataDesc desc; + if ((desc = tryGetFromOtherThread()) != null) { + submitMore(); + return validateLastEntryAndGetBuf(desc.getEntry(entryIdx)); + } + // duplicate and get data by self + return readDuplicated(true); + } catch (InterruptedException e) { + throw new IOException("read interrupted.", e); + } + } + + /** + * we have to duplicate submitted desc since mapId was moved. + * + * @return + * @throws IOException + */ + private ByteBuf readDuplicated(boolean expectNotNullCtx) throws IOException { + DaosReader.ReadTaskContext context = getNextNonReturnedCtx(); + if (context == null) { + if (expectNotNullCtx) { + throw new IllegalStateException("context should not be null. totalSubmitted: " + totalSubmitted); + } + if (!fromOtherThread) { + lastCtx = null; + } + return null; + } + IODataDesc newDesc = context.getDesc().duplicate(); + ByteBuf buf = getBySelf(newDesc, context.getMapReduceId()); + selfCurrentCtx = context; + return buf; + } + + @Override + protected DaosReader.ReadTaskContext getNextNonReturnedCtx() { + // in case no even single return from other thread + // check selfCurrentCtx since the wait could span multiple contexts/descs + DaosReader.ReadTaskContext curCtx = selfCurrentCtx == null ? + getCurrentCtx() : selfCurrentCtx; + if (curCtx == null) { + return getHeadCtx(); + } + // no consumedStack push and no totalInMemSize and totalSubmitted update + // since they will be updated when the task context finally returned + return curCtx.getNext(); + } + + private IODataDesc tryGetFromOtherThread() throws InterruptedException, IOException { + IODataDesc desc = tryGetValidCompleted(); + if (desc != null) { + return desc; + } + // check completion + if ((!mapIdIt.hasNext()) && curMapReduceId == null && totalSubmitted == 0) { + return null; + } + // wait for specified time + desc = waitForValidFromOtherThread(); + if (desc != null) { + return desc; + } + // check wait times and cancel task + // TODO: stop reading from other threads? + cancelTasks(false); + return null; + } + + private IODataDesc waitForValidFromOtherThread() throws InterruptedException, IOException { + IODataDesc desc; + while (true) { + long start = System.nanoTime(); + boolean timeout = waitForCondition(config.waitDataTimeMs); + metrics.incFetchWaitTime(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start)); + if (timeout) { + exceedWaitTimes++; + if (log.isDebugEnabled()) { + log.debug("exceed wait: {}ms, times: {}", config.waitDataTimeMs, exceedWaitTimes); + } + if (exceedWaitTimes >= config.waitTimeoutTimes) { + return null; + } + } + // get some results after wait + desc = tryGetValidCompleted(); + if (desc != null) { + return desc; + } + } + } + + protected IODataDesc tryGetValidCompleted() throws IOException { + if (moveForward()) { + return currentDesc; + } + return null; + } + + @Override + protected boolean consumed(LinkedTaskContext consumed) { + return !consumed.isCancelled(); + } + + @Override + protected boolean validateReturned(LinkedTaskContext context) throws IOException { + if (context.isCancelled()) { + return false; + } + selfCurrentCtx = null; // non-cancelled currentCtx overrides selfCurrentCtx + lastMapReduceIdForReturn = ((DaosReader.ReadTaskContext)context).getMapReduceId(); + IODataDesc desc = context.getDesc(); + if (!desc.isSucceeded()) { + String msg = "failed to get data from DAOS, desc: " + desc.toString(4096); + if (desc.getCause() != null) { + throw new IOException(msg, desc.getCause()); + } else { + throw new IllegalStateException(msg + "\nno exception got. logic error or crash?"); + } + } + currentDesc = desc; + return true; + } + + private ByteBuf tryCurrentDesc() throws IOException { + if (currentDesc != null) { + ByteBuf buf; + while (entryIdx < currentDesc.getNbrOfEntries()) { + IODataDesc.Entry entry = currentDesc.getEntry(entryIdx); + buf = validateLastEntryAndGetBuf(entry); + if (buf.readableBytes() > 0) { + return buf; + } + entryIdx++; + } + entryIdx = 0; + // no need to release desc since all its entries are released in tryCurrentEntry and + // internal buffers are released after object.fetch + // reader.close will release all in case of failure + currentDesc = null; + } + return null; + } + + private ByteBuf tryCurrentEntry() { + if (currentEntry != null && !currentEntry.isFetchBufReleased()) { + ByteBuf buf = currentEntry.getFetchedData(); + if (buf.readableBytes() > 0) { + return buf; + } + // release buffer as soon as possible + currentEntry.releaseDataBuffer(); + entryIdx++; + } + // not null currentEntry since it will be used for size validation + return null; + } + + /** + * for first read. + * + * @param selfReadLimit + * @return + * @throws IOException + */ + private ByteBuf getBySelfAndSubmitMore(long selfReadLimit) throws IOException { + entryIdx = 0; + // fetch the next by self + IODataDesc desc = createNextDesc(selfReadLimit); + Tuple2 mapreduceId = lastMapReduceIdForSubmit; + try { + if (fromOtherThread) { + submitMore(); + } + } catch (Exception e) { + desc.release(); + if (e instanceof IOException) { + throw (IOException)e; + } + throw new IOException("failed to submit more", e); + } + // first time read from reduce task + return getBySelf(desc, mapreduceId); + } + + private void submitMore() throws IOException { + while (totalSubmitted < config.readBatchSize && totalInMemSize < config.maxMem) { + IODataDesc taskDesc = createNextDesc(config.maxBytesInFlight); + if (taskDesc == null) { + break; + } + submit(taskDesc, lastMapReduceIdForSubmit); + } + } + + @Override + protected Runnable newTask(LinkedTaskContext context) { + return DaosReader.ReadTask.newInstance((DaosReader.ReadTaskContext) context); + } + + @Override + protected LinkedTaskContext createTaskContext(IODataDesc desc, Object morePara) { + return new DaosReader.ReadTaskContext(object, counter, lock, condition, desc, morePara); + } + + private ByteBuf getBySelf(IODataDesc desc, Tuple2 mapreduceId) throws IOException { + // get data by self, no need to release currentDesc + if (desc == null) { // reach end + return null; + } + boolean releaseBuf = false; + try { + object.fetch(desc); + currentDesc = desc; + ByteBuf buf = validateLastEntryAndGetBuf(desc.getEntry(entryIdx)); + lastMapReduceIdForReturn = mapreduceId; + return buf; + } catch (IOException | IllegalStateException e) { + releaseBuf = true; + throw e; + } finally { + desc.release(releaseBuf); + } + } + + private IODataDesc createNextDesc(long sizeLimit) throws IOException { + long remaining = sizeLimit; + int reduceId = -1; + long mapId; + IODataDesc desc = null; + while (remaining > 0) { + nextMapReduceId(); + if (curMapReduceId == null) { + break; + } + if (reduceId > 0 && curMapReduceId._2 != reduceId) { // make sure entries under same reduce + break; + } + reduceId = curMapReduceId._2; + mapId = curMapReduceId._1; + lastMapReduceIdForSubmit = curMapReduceId; + long readSize = partSizeMap.get(curMapReduceId)._1() - curOffset; + long offset = curOffset; + if (readSize > remaining) { + readSize = remaining; + curOffset += readSize; + } else { + curOffset = 0; + curMapReduceId = null; + } + if (desc == null) { + desc = object.createDataDescForFetch(String.valueOf(reduceId), IODataDesc.IodType.ARRAY, 1); + } + desc.addEntryForFetch(String.valueOf(mapId), (int)offset, (int)readSize); + remaining -= readSize; + } + return desc; + } + + private void nextMapReduceId() { + if (curMapReduceId != null) { + return; + } + curOffset = 0; + if (mapIdIt.hasNext()) { + curMapReduceId = mapIdIt.next(); + partsRead++; + } else { + curMapReduceId = null; + } + } + + private ByteBuf validateLastEntryAndGetBuf(IODataDesc.Entry entry) throws IOException { + ByteBuf buf = entry.getFetchedData(); + int byteLen = buf.readableBytes(); + newMap = false; + if (currentEntry != null && entry != currentEntry) { + if (entry.getKey().equals(currentEntry.getKey())) { + currentPartSize += byteLen; + } else { + checkPartitionSize(); + newMap = true; + currentPartSize = byteLen; + } + } + currentEntry = entry; + metrics.incRemoteBytesRead(byteLen); + return buf; + } + + private void checkPartitionSize() throws IOException { + if (lastMapReduceIdForReturn == null) { + return; + } + // partition size is not accurate after compress/decompress + long size = partSizeMap.get(lastMapReduceIdForReturn)._1(); + if (size < 35 * 1024 * 1024 * 1024 && currentPartSize * 1.1 < size) { + throw new IOException("expect partition size " + partSizeMap.get(lastMapReduceIdForReturn) + + ", actual size " + currentPartSize + ", mapId and reduceId: " + lastMapReduceIdForReturn); + } + metrics.incRemoteBlocksFetched(1); + } + + public boolean cleanup(boolean force) { + boolean allReleased = true; + if (!cleaned) { + allReleased &= cleanupSubmitted(force); + allReleased &= cleanupConsumed(force); + } + return allReleased; + } + + public void checkTotalPartitions() throws IOException { + if (partsRead != totalParts) { + throw new IOException("expect total partitions to be read: " + totalParts + ", actual read: " + partsRead); + } + } + + @Override + public DaosReader.ReadTaskContext getCurrentCtx() { + return (DaosReader.ReadTaskContext) currentCtx; + } + + @Override + public DaosReader.ReadTaskContext getHeadCtx() { + return (DaosReader.ReadTaskContext) headCtx; + } + + @Override + public DaosReader.ReadTaskContext getLastCtx() { + return (DaosReader.ReadTaskContext) lastCtx; + } + } + + /** + * reader configurations, please check configs prefixed with SHUFFLE_DAOS_READ in {@link package$#MODULE$}. + */ + private static final class ReaderConfig { + private long minReadSize; + private long maxBytesInFlight; + private long maxMem; + private int readBatchSize; + private int waitDataTimeMs; + private int waitTimeoutTimes; + private boolean fromOtherThread; + + private ReaderConfig(long maxBytesInFlight, long maxMem) { + SparkConf conf = SparkEnv.get().conf(); + minReadSize = (long)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_MINIMUM_SIZE()) * 1024; + if (maxBytesInFlight < minReadSize) { + this.maxBytesInFlight = minReadSize; + } else { + this.maxBytesInFlight = maxBytesInFlight; + } + this.maxMem = maxMem; + this.readBatchSize = (int)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_BATCH_SIZE()); + this.waitDataTimeMs = (int)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_WAIT_DATA_MS()); + this.waitTimeoutTimes = (int)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_WAIT_DATA_TIMEOUT_TIMES()); + this.fromOtherThread = (boolean)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_FROM_OTHER_THREAD()); + if (log.isDebugEnabled()) { + log.debug("minReadSize: " + minReadSize); + log.debug("maxBytesInFlight: " + maxBytesInFlight); + log.debug("maxMem: " + maxMem); + log.debug("readBatchSize: " + readBatchSize); + log.debug("waitDataTimeMs: " + waitDataTimeMs); + log.debug("waitTimeoutTimes: " + waitTimeoutTimes); + log.debug("fromOtherThread: " + fromOtherThread); + } + } + } +} diff --git a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleOutputStream.java b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleOutputStream.java new file mode 100644 index 00000000..bda17ea3 --- /dev/null +++ b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleOutputStream.java @@ -0,0 +1,75 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos; + +import java.io.IOException; +import java.io.OutputStream; + +/** + * OutputStream wrapper of {@link DaosWriter} for each map partition + */ +public class DaosShuffleOutputStream extends OutputStream { + + private int partitionId; + private DaosWriter daosWriter; + + private long writtenBytes = 0L; + + public DaosShuffleOutputStream(int partitionId, DaosWriter daosWriter) { + this.partitionId = partitionId; + this.daosWriter = daosWriter; + } + + @Override + public void write(int b) { + daosWriter.write(partitionId, (byte)b); + writtenBytes += 1; + } + + @Override + public void write(byte[] b) { + daosWriter.write(partitionId, b); + writtenBytes += b.length; + } + + @Override + public void write(byte[] b, int off, int len) { + daosWriter.write(partitionId, b, off, len); + writtenBytes += len; + } + + @Override + public void flush() throws IOException { + // do nothing since we want to control the actual DAOS write, not objectoutputstream/kryo + } + + @Override + public void close() throws IOException { + daosWriter.flush(partitionId); + } + + public long getWrittenBytes() { + return writtenBytes; + } +} diff --git a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosWriter.java b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosWriter.java new file mode 100644 index 00000000..8b9714f2 --- /dev/null +++ b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosWriter.java @@ -0,0 +1,690 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos; + +import io.daos.BufferAllocator; +import io.daos.DaosIOException; +import io.daos.obj.DaosObject; +import io.daos.obj.IODataDesc; +import io.netty.buffer.ByteBuf; +import io.netty.util.internal.ObjectPool; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.*; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; + +/** + * A DAOS writer per map task which may have multiple map output partitions. + * Each partition has one corresponding {@link NativeBuffer} which caches records until + * a specific {@link #flush(int)} call being made. Then {@link NativeBuffer} creates + * {@link IODataDesc} and write to DAOS in either caller thread or other dedicated thread. + */ +public class DaosWriter extends TaskSubmitter { + + private DaosObject object; + + private String mapId; + + private WriteParam param; + + private WriteConfig config; + + private Map writerMap; + + private NativeBuffer[] partitionBufArray; + + private int totalTimeoutTimes; + + private int totalWriteTimes; + + private int totalBySelfTimes; + + private volatile boolean cleaned; + + private static Logger LOG = LoggerFactory.getLogger(DaosWriter.class); + + /** + * construct DaosWriter with object and dedicated read executors. + * + * @param param + * write parameters + * @param object + * opened DaosObject + * @param executor + * null means write in caller's thread. Submit {@link WriteTask} to it otherwise. + */ + public DaosWriter(DaosWriter.WriteParam param, DaosObject object, + BoundThreadExecutors.SingleThreadExecutor executor) { + super(executor); + this.param = param; + this.config = param.config; + this.partitionBufArray = new NativeBuffer[param.numPartitions]; + this.mapId = String.valueOf(param.mapId); + this.object = object; + } + + private NativeBuffer getNativeBuffer(int partitionId) { + NativeBuffer buffer = partitionBufArray[partitionId]; + if (buffer == null) { + buffer = new NativeBuffer(partitionId, config.bufferSize); + partitionBufArray[partitionId] = buffer; + } + return buffer; + } + + /** + * write to buffer. + * + * @param partitionId + * @param b + */ + public void write(int partitionId, int b) { + getNativeBuffer(partitionId).write(b); + } + + /** + * write to buffer. + * + * @param partitionId + * @param array + */ + public void write(int partitionId, byte[] array) { + getNativeBuffer(partitionId).write(array); + } + + /** + * write to buffer. + * + * @param partitionId + * @param array + * @param offset + * @param len + */ + public void write(int partitionId, byte[] array, int offset, int len) { + getNativeBuffer(partitionId).write(array, offset, len); + } + + /** + * get length of all partitions. + * 0 for empty partition. + * + * @param numPartitions + * @return array of partition lengths + */ + public long[] getPartitionLens(int numPartitions) { + if (LOG.isDebugEnabled()) { + LOG.debug("partition map size: " + partitionBufArray.length); + for (int i = 0; i < numPartitions; i++) { + NativeBuffer nb = partitionBufArray[i]; + if (nb != null) { + LOG.debug("id: " + i + ", native buffer: " + nb.partitionId + ", " + + nb.totalSize + ", " + nb.roundSize); + } + } + } + long[] lens = new long[numPartitions]; + for (int i = 0; i < numPartitions; i++) { + NativeBuffer nb = partitionBufArray[i]; + if (nb != null) { + lens[i] = nb.totalSize; + if (nb.roundSize != 0 || !nb.bufList.isEmpty()) { + throw new IllegalStateException("round size should be 0, " + nb.roundSize + ", buflist should be empty, " + + nb.bufList.size()); + } + } else { + lens[i] = 0; + } + } + return lens; + } + + /** + * Flush specific partition to DAOS. + * + * @param partitionId + * @throws IOException + */ + public void flush(int partitionId) throws IOException { + NativeBuffer buffer = partitionBufArray[partitionId]; + if (buffer == null) { + return; + } + IODataDesc desc = buffer.createUpdateDesc(); + if (desc == null) { + return; + } + totalWriteTimes++; + if (config.warnSmallWrite && buffer.roundSize < config.minSize) { + LOG.warn("too small partition size {}, shuffle {}, map {}, partition {}", + buffer.roundSize, param.shuffleId, mapId, partitionId); + } + if (executor == null) { // run write by self + runBySelf(desc, buffer); + return; + } + submitToOtherThreads(desc, buffer); + } + + private void runBySelf(IODataDesc desc, NativeBuffer buffer) throws IOException { + totalBySelfTimes++; + try { + object.update(desc); + } catch (IOException e) { + throw new IOException("failed to write partition of " + desc, e); + } finally { + desc.release(); + buffer.reset(true); + } + } + + private void submitToOtherThreads(IODataDesc desc, NativeBuffer buffer) throws IOException { + // move forward to release write buffers + moveForward(); + // check if we need to wait submitted tasks to be executed + if (goodForSubmit()) { + submitAndReset(desc, buffer); + return; + } + // to wait + int timeoutTimes = 0; + try { + while (!goodForSubmit()) { + boolean timeout = waitForCondition(config.waitTimeMs); + moveForward(); + if (timeout) { + timeoutTimes++; + if (LOG.isDebugEnabled()) { + LOG.debug("wait daos write timeout times: " + timeoutTimes); + } + if (timeoutTimes >= config.timeoutTimes) { + totalTimeoutTimes += timeoutTimes; + runBySelf(desc, buffer); + return; + } + } + } + } catch (InterruptedException e) { + desc.release(); + Thread.currentThread().interrupt(); + throw new IOException("interrupted when wait daos write", e); + } + // submit write task after some wait + totalTimeoutTimes += timeoutTimes; + submitAndReset(desc, buffer); + } + + private boolean goodForSubmit() { + return totalInMemSize < config.totalInMemSize && totalSubmitted < config.totalSubmittedLimit; + } + + private void submitAndReset(IODataDesc desc, NativeBuffer buffer) { + try { + submit(desc, buffer.bufList); + } finally { + buffer.reset(false); + } + } + + private void cleanup(boolean force) { + if (cleaned) { + return; + } + boolean allReleased = true; + allReleased &= cleanupSubmitted(force); + allReleased &= cleanupConsumed(force); + if (allReleased) { + cleaned = true; + } + } + + /** + * wait write task to be completed and clean up resources. + */ + public void close() { + try { + close(true); + } catch (Exception e) { + throw new IllegalStateException("failed to complete all write tasks and cleanup", e); + } + } + + private void close(boolean force) throws Exception { + if (partitionBufArray != null) { + waitCompletion(force); + partitionBufArray = null; + object = null; + if (LOG.isDebugEnabled()) { + LOG.debug("total writes: " + totalWriteTimes + ", total timeout times: " + totalTimeoutTimes + + ", total write-by-self times: " + totalBySelfTimes + ", total timeout times/total writes: " + + ((float) totalTimeoutTimes) / totalWriteTimes); + } + } + cleanup(force); + if (writerMap != null && (force || cleaned)) { + writerMap.remove(this); + writerMap = null; + } + } + + private void waitCompletion(boolean force) throws Exception { + if (!force) { + return; + } + try { + while (totalSubmitted > 0) { + waitForCondition(config.waitTimeMs); + moveForward(); + } + } catch (Exception e) { + LOG.error("failed to wait completion of daos writing", e); + throw e; + } + } + + public void setWriterMap(Map writerMap) { + writerMap.put(this, 0); + this.writerMap = writerMap; + } + + @Override + protected Runnable newTask(LinkedTaskContext context) { + return WriteTask.newInstance((WriteTaskContext) context); + } + + @Override + protected LinkedTaskContext createTaskContext(IODataDesc desc, Object morePara) { + return new WriteTaskContext(object, counter, lock, condition, desc, morePara); + } + + @Override + protected boolean validateReturned(LinkedTaskContext context) throws IOException { + if (!context.desc.isSucceeded()) { + throw new DaosIOException("write is not succeeded: " + context.desc); + } + return false; + } + + @Override + protected boolean consumed(LinkedTaskContext context) { + // release write buffers + List bufList = (List) context.morePara; + bufList.forEach(b -> b.release()); + bufList.clear(); + return true; + } + + /** + * Write data to one or multiple netty direct buffers which will be written to DAOS without copy + */ + private class NativeBuffer implements Comparable { + private int partitionId; + private String partitionIdKey; + private int bufferSize; + private int idx = -1; + private List bufList = new ArrayList<>(); + private long totalSize; + private long roundSize; + + NativeBuffer(int partitionId, int bufferSize) { + this.partitionId = partitionId; + this.partitionIdKey = String.valueOf(partitionId); + this.bufferSize = bufferSize; + } + + private ByteBuf addNewByteBuf(int len) { + ByteBuf buf; + try { + buf = BufferAllocator.objBufWithNativeOrder(Math.max(bufferSize, len)); + } catch (OutOfMemoryError e) { + LOG.error("too big buffer size: " + Math.max(bufferSize, len)); + throw e; + } + bufList.add(buf); + idx++; + return buf; + } + + private ByteBuf getBuffer(int len) { + if (idx < 0) { + return addNewByteBuf(len); + } + return bufList.get(idx); + } + + public void write(int b) { + ByteBuf buf = getBuffer(1); + if (buf.writableBytes() < 1) { + buf = addNewByteBuf(1); + } + buf.writeByte(b); + roundSize += 1; + } + + public void write(byte[] b) { + write(b, 0, b.length); + } + + public void write(byte[] b, int offset, int len) { + if (len <= 0) { + return; + } + ByteBuf buf = getBuffer(len); + int avail = buf.writableBytes(); + int gap = len - avail; + if (gap <= 0) { + buf.writeBytes(b, offset, len); + } else { + buf.writeBytes(b, offset, avail); + buf = addNewByteBuf(gap); + buf.writeBytes(b, avail, gap); + } + roundSize += len; + } + + public IODataDesc createUpdateDesc() throws IOException { + if (roundSize == 0 || bufList.isEmpty()) { + return null; + } + long bufSize = 0; + IODataDesc desc = object.createDataDescForUpdate(partitionIdKey, IODataDesc.IodType.ARRAY, 1); + for (ByteBuf buf : bufList) { + desc.addEntryForUpdate(mapId, (int) totalSize, buf); + bufSize += buf.readableBytes(); + } + if (roundSize != bufSize) { + throw new IOException("expect update size: " + roundSize + ", actual: " + bufSize); + } + return desc; + } + + public void reset(boolean release) { + if (release) { + bufList.forEach(b -> b.release()); + } + // release==false, buffers will be released when tasks are executed and consumed + bufList.clear(); + idx = -1; + totalSize += roundSize; + roundSize = 0; + } + + @Override + public int compareTo(NativeBuffer nativeBuffer) { + return partitionId - nativeBuffer.partitionId; + } + } + + /** + * Write configurations. Please check configs prefixed with SHUFFLE_DAOS_WRITE in {@link package$#MODULE$}. + */ + public static class WriteConfig { + private int bufferSize; + private int minSize; + private boolean warnSmallWrite; + private long waitTimeMs; + private int timeoutTimes; + private long totalInMemSize; + private int totalSubmittedLimit; + private int threads; + private boolean fromOtherThreads; + + public WriteConfig bufferSize(int bufferSize) { + this.bufferSize = bufferSize; + return this; + } + + public WriteConfig minSize(int minSize) { + this.minSize = minSize; + return this; + } + + public WriteConfig warnSmallWrite(boolean warnSmallWrite) { + this.warnSmallWrite = warnSmallWrite; + return this; + } + + public WriteConfig waitTimeMs(long waitTimeMs) { + this.waitTimeMs = waitTimeMs; + return this; + } + + public WriteConfig timeoutTimes(int timeoutTimes) { + this.timeoutTimes = timeoutTimes; + return this; + } + + public WriteConfig totalInMemSize(long totalInMemSize) { + this.totalInMemSize = totalInMemSize; + return this; + } + + public WriteConfig totalSubmittedLimit(int totalSubmittedLimit) { + this.totalSubmittedLimit = totalSubmittedLimit; + return this; + } + + public WriteConfig threads(int threads) { + this.threads = threads; + return this; + } + + public WriteConfig fromOtherThreads(boolean fromOtherThreads) { + this.fromOtherThreads = fromOtherThreads; + return this; + } + + public int getBufferSize() { + return bufferSize; + } + + public int getMinSize() { + return minSize; + } + + public boolean isWarnSmallWrite() { + return warnSmallWrite; + } + + public long getWaitTimeMs() { + return waitTimeMs; + } + + public int getTimeoutTimes() { + return timeoutTimes; + } + + public long getTotalInMemSize() { + return totalInMemSize; + } + + public int getTotalSubmittedLimit() { + return totalSubmittedLimit; + } + + public int getThreads() { + return threads; + } + + public boolean isFromOtherThreads() { + return fromOtherThreads; + } + + @Override + public String toString() { + return "WriteConfig{" + + "bufferSize=" + bufferSize + + ", minSize=" + minSize + + ", warnSmallWrite=" + warnSmallWrite + + ", waitTimeMs=" + waitTimeMs + + ", timeoutTimes=" + timeoutTimes + + ", totalInMemSize=" + totalInMemSize + + ", totalSubmittedLimit=" + totalSubmittedLimit + + ", threads=" + threads + + ", fromOtherThreads=" + fromOtherThreads + + '}'; + } + } + + public static class WriteParam { + private int numPartitions; + private int shuffleId; + private long mapId; + private WriteConfig config; + + public WriteParam numPartitions(int numPartitions) { + this.numPartitions = numPartitions; + return this; + } + + public WriteParam shuffleId(int shuffleId) { + this.shuffleId = shuffleId; + return this; + } + + public WriteParam mapId(long mapId) { + this.mapId = mapId; + return this; + } + + public WriteParam config(WriteConfig config) { + this.config = config; + return this; + } + } + + /** + * Task to write data to DAOS. Task itself is cached to reduce GC time. + * To reuse task for different writes, prepare and reset {@link WriteTaskContext} by calling + * {@link #newInstance(WriteTaskContext)} + */ + static final class WriteTask implements Runnable { + private final ObjectPool.Handle handle; + private WriteTaskContext context; + + private static final ObjectPool objectPool = ObjectPool.newPool(handle -> new WriteTask(handle)); + + private static final Logger log = LoggerFactory.getLogger(WriteTask.class); + + static WriteTask newInstance(WriteTaskContext context) { + WriteTask task = objectPool.get(); + task.context = context; + return task; + } + + private WriteTask(ObjectPool.Handle handle) { + this.handle = handle; + } + + @Override + public void run() { + boolean cancelled = context.cancelled; + try { + if (!cancelled) { + context.object.update(context.desc); + } + } catch (Exception e) { + log.error("failed to write for " + context.desc, e); + } finally { + context.desc.release(); + context.signal(); + context = null; + handle.recycle(this); + } + } + } + + /** + * Context for write task. It holds all other object to read and sync between caller thread and write thread. + * It should be cached in caller thread for reusing. + */ + static final class WriteTaskContext extends LinkedTaskContext { + + /** + * constructor with all parameters. Some of them can be reused later. + * + * @param object + * DAOS object to fetch data from DAOS + * @param counter + * counter to indicate how many write is on-going + * @param writeLock + * lock to work with notFull condition to signal caller thread to submit more write task + * @param notFull + * condition to signal caller thread + * @param desc + * desc object to describe where to write data + * @param bufList + * list of buffers to write to DAOS + */ + WriteTaskContext(DaosObject object, AtomicInteger counter, Lock writeLock, Condition notFull, + IODataDesc desc, Object bufList) { + super(object, counter, writeLock, notFull); + this.desc = desc; + List myBufList = new ArrayList<>(); + myBufList.addAll((List) bufList); + this.morePara = myBufList; + } + + @Override + public WriteTaskContext getNext() { + return (WriteTaskContext) next; + } + + @Override + public void reuse(IODataDesc desc, Object morePara) { + List myBufList = (List) this.morePara; + if (!myBufList.isEmpty()) { + throw new IllegalStateException("bufList in reusing write task context should be empty"); + } + myBufList.addAll((List) morePara); + super.reuse(desc, myBufList); + } + } + + /** + * Thread factory for write + */ + protected static class WriteThreadFactory implements ThreadFactory { + private AtomicInteger id = new AtomicInteger(0); + + @Override + public Thread newThread(Runnable runnable) { + Thread t; + String name = "daos_write_" + id.getAndIncrement(); + if (runnable == null) { + t = new Thread(name); + } else { + t = new Thread(runnable, name); + } + t.setDaemon(true); + t.setUncaughtExceptionHandler((thread, throwable) -> + LOG.error("exception occurred in thread " + name, throwable)); + return t; + } + } +} diff --git a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/LinkedTaskContext.java b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/LinkedTaskContext.java new file mode 100644 index 00000000..9a8f345d --- /dev/null +++ b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/LinkedTaskContext.java @@ -0,0 +1,143 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos; + +import io.daos.obj.DaosObject; +import io.daos.obj.IODataDesc; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; + +/** + * Linked reusable task context for read and write. + * Task context acts like a interface between caller thread and task executor thread. + * It wraps task parameters, synchronization blocks and links. + */ +public abstract class LinkedTaskContext { + + protected final DaosObject object; + protected final AtomicInteger counter; + protected final Lock lock; + protected final Condition condition; + + protected IODataDesc desc; + protected LinkedTaskContext next; + + protected volatile boolean cancelled; // for multi-thread + protected boolean cancelledByCaller; // for accessing by caller + + protected Object morePara; + + private static final Logger logger = LoggerFactory.getLogger(LinkedTaskContext.class); + + /** + * constructor with parameters can be reused for all tasks. + * + * @param object + * daos object for either read or write + * @param counter + * counter to indicate how many data is ready to be consumed or produced + * @param lock + * lock to work with condition to signal caller thread there is data ready + * @param condition + * condition to signal caller thread + */ + protected LinkedTaskContext(DaosObject object, AtomicInteger counter, Lock lock, Condition condition) { + this.object = object; + this.counter = counter; + this.lock = lock; + this.condition = condition; + } + + /** + * reuse this context by setting task specific data and resetting some status. + * + * @param desc + * data description + * @param morePara + * additional data + */ + protected void reuse(IODataDesc desc, Object morePara) { + this.desc = desc; + this.next = null; + this.morePara = morePara; + cancelled = false; + cancelledByCaller = false; + } + + /** + * link task context. + * + * @param next + */ + protected void setNext(LinkedTaskContext next) { + this.next = next; + } + + protected LinkedTaskContext getNext() { + return next; + } + + protected IODataDesc getDesc() { + return desc; + } + + /** + * signal caller thread on condition + */ + protected void signal() { + counter.getAndIncrement(); + try { + lock.lockInterruptibly(); + try { + condition.signal(); + } finally { + lock.unlock(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + logger.error("interrupted when signal task completion for " + desc, e); + } + } + + /** + * cancel task + */ + public void cancel() { + cancelled = true; + cancelledByCaller = true; + } + + /** + * check if task is cancelled. It's for caller thread to avoid volatile access. + * + * @return true for cancelled. false otherwise. + */ + public boolean isCancelled() { + return cancelledByCaller; + } +} diff --git a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/TaskSubmitter.java b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/TaskSubmitter.java new file mode 100644 index 00000000..b03d3a60 --- /dev/null +++ b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/TaskSubmitter.java @@ -0,0 +1,304 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos; + +import io.daos.obj.IODataDesc; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Deque; +import java.util.LinkedList; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +/** + * Parent class to submit task to {@link org.apache.spark.shuffle.daos.BoundThreadExecutors.SingleThreadExecutor}. + */ +public abstract class TaskSubmitter { + + protected final BoundThreadExecutors.SingleThreadExecutor executor; + + protected LinkedTaskContext headCtx; + protected LinkedTaskContext currentCtx; + protected LinkedTaskContext lastCtx; + protected Deque consumedStack = new LinkedList<>(); + + protected Lock lock = new ReentrantLock(); + protected Condition condition = lock.newCondition(); + protected AtomicInteger counter = new AtomicInteger(0); + + protected int exceedWaitTimes; + protected long totalInMemSize; + protected int totalSubmitted; + + private static final Logger log = LoggerFactory.getLogger(TaskSubmitter.class); + + protected TaskSubmitter(BoundThreadExecutors.SingleThreadExecutor executor) { + this.executor = executor; + } + + /** + * submit read task with taskDesc and morePara. + * + * @param taskDesc + * IO description object + * @param morePara + * additional parameter + */ + protected void submit(IODataDesc taskDesc, Object morePara) { + LinkedTaskContext context = tryReuseContext(taskDesc, morePara); + executor.execute(newTask(context)); + totalInMemSize += taskDesc.getTotalRequestSize(); + totalSubmitted++; + } + + protected abstract Runnable newTask(LinkedTaskContext context); + + /** + * wait for condition to be met. + * + * @param waitDataTimeMs + * @return true for timeout, false otherwise. + * @throws {@link InterruptedException} + */ + protected boolean waitForCondition(long waitDataTimeMs) throws InterruptedException { + lock.lockInterruptibly(); + try { + if (!condition.await(waitDataTimeMs, TimeUnit.MILLISECONDS)) { + return true; + } + return false; + } finally { + lock.unlock(); + } + } + + /** + * reuse task context in caller thread. + * + * @param desc + * desc object to override old desc in reused task context + * @param morePara + * additional parameter to override old desc in reused task context + * @return reused linked task context + */ + protected LinkedTaskContext tryReuseContext(IODataDesc desc, Object morePara) { + LinkedTaskContext context = consumedStack.poll(); + if (context != null) { + context.reuse(desc, morePara); + } else { + context = createTaskContext(desc, morePara); + } + if (lastCtx != null) { + lastCtx.setNext(context); + } + lastCtx = context; + if (headCtx == null) { + headCtx = context; + } + return context; + } + + /** + * create new task context if there is no for reusing. + * + * @param desc + * desc object + * @param morePara + * additional parameter + * @return Linked task context + */ + protected abstract LinkedTaskContext createTaskContext(IODataDesc desc, Object morePara); + + /** + * move forward by checking if there is any returned task available. + * consume and validate each of them. + * + * @return true if we consume some task and {@link #consume()} returns true. false otherwise. + * @throws IOException + */ + protected boolean moveForward() throws IOException { + int c; + while ((c = counter.decrementAndGet()) >= 0) { + if (consume()) { + return true; + } + } + if (c < 0) { + if (log.isDebugEnabled()) { + log.debug("spurious wakeup"); + } + counter.incrementAndGet(); + } + return false; + } + + /** + * consume returned task. + * + * @return true to stop consuming more returned task. false to continue. + * @throws IOException + */ + private boolean consume() throws IOException { + if (currentCtx != null) { + totalInMemSize -= currentCtx.getDesc().getTotalRequestSize(); + if (consumed(currentCtx)) { // check if consumed context can be reused + consumedStack.offer(currentCtx); + } + currentCtx = currentCtx.getNext(); + } else { // returned first time + currentCtx = headCtx; + } + totalSubmitted -= 1; + return validateReturned(currentCtx); + } + + /** + * validate if returned task is ok. + * + * @param context + * task context + * @return true to stop consuming more returned task. false to continue. + * @throws IOException + */ + protected abstract boolean validateReturned(LinkedTaskContext context) throws IOException; + + /** + * post action after task data being consumed. + * + * @param context + * task context + * @return true for reusing consumed task context. false means it cannot be reused. + */ + protected abstract boolean consumed(LinkedTaskContext context); + + /** + * cancel tasks. + * + * @param cancelAll + * true to cancel all tasks. false to cancel just one task. + */ + protected void cancelTasks(boolean cancelAll) { + LinkedTaskContext ctx = getNextNonReturnedCtx(); + if (ctx == null) { // reach to end + return; + } + ctx.cancel(); + if (cancelAll) { + ctx = ctx.getNext(); + while (ctx != null) { + ctx.cancel(); + ctx = ctx.getNext(); + } + } + } + + /** + * get next task context sequentially without waiting signal from other threads. + * + * @return task context + */ + protected LinkedTaskContext getNextNonReturnedCtx() { + LinkedTaskContext ctx = currentCtx; + if (ctx == null) { + return getHeadCtx(); + } + return ctx.getNext(); + } + + /** + * cancel and clean up all submitted task contexts. + * + * @param force + * force cleanup? + * @return true if all is cleaned up. false otherwise. + */ + protected boolean cleanupSubmitted(boolean force) { + boolean allReleased = true; + LinkedTaskContext ctx = currentCtx; + while (ctx != null) { + if (!ctx.isCancelled()) { + ctx.cancel(); + } + allReleased &= cleanupTaskContext(ctx, force); + ctx = ctx.getNext(); + } + return allReleased; + } + + /** + * clean up all linked consumed task contexts. + * + * @param force + * force cleanup? + * @return true if all is cleaned up. false otherwise. + */ + protected boolean cleanupConsumed(boolean force) { + boolean allReleased = true; + for (LinkedTaskContext ctx : consumedStack) { + allReleased &= cleanupTaskContext(ctx, force); + } + if (allReleased) { + consumedStack.clear(); + } + return allReleased; + } + + /** + * clean up task context. + * + * @param ctx + * task context + * @param force + * force close? + * @return true for successful cleanup, false otherwise + */ + protected boolean cleanupTaskContext(LinkedTaskContext ctx, boolean force) { + if (ctx == null) { + return true; + } + IODataDesc desc = ctx.getDesc(); + if (desc != null && (force || desc.isSucceeded())) { + desc.release(); + return true; + } + return false; + } + + public LinkedTaskContext getCurrentCtx() { + return currentCtx; + } + + public LinkedTaskContext getHeadCtx() { + return headCtx; + } + + public LinkedTaskContext getLastCtx() { + return lastCtx; + } +} diff --git a/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/DaosShuffleManager.scala b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/DaosShuffleManager.scala new file mode 100644 index 00000000..8614b791 --- /dev/null +++ b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/DaosShuffleManager.scala @@ -0,0 +1,181 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos + +import java.lang.reflect.Method +import java.util.concurrent.ConcurrentHashMap + +import io.daos.DaosClient +import scala.collection.JavaConverters._ + +import org.apache.spark.{ShuffleDependency, SparkConf, SparkEnv, TaskContext} +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.shuffle._ +import org.apache.spark.shuffle.sort.SortShuffleManager.canUseBatchFetch +import org.apache.spark.util.{ShutdownHookManager, Utils} +import org.apache.spark.util.collection.OpenHashSet + +/** + * A shuffle manager to write and read map data from DAOS using DAOS object API. + * + * @param conf + * spark configuration + */ +class DaosShuffleManager(conf: SparkConf) extends ShuffleManager with Logging { + + logInfo("loaded " + classOf[DaosShuffleManager]) + + if (conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) { + throw new IllegalArgumentException("DaosShuffleManager doesn't support old fetch protocol. Please remove " + + config.SHUFFLE_USE_OLD_FETCH_PROTOCOL.key) + } + + def findHadoopFs: Method = { + try { + val fsClass = Utils.classForName("org.apache.hadoop.fs.FileSystem") + fsClass.getMethod("closeAll") + } catch { + case _: Throwable => null + } + } + + val closeAllFsMethod = findHadoopFs + + def closeAllHadoopFs: Unit = { + if (closeAllFsMethod == null) { + return + } + try { + closeAllFsMethod.invoke(null) + } catch { + case _: Throwable => // ignore all exceptions + } + } + + val daosShuffleIO = new DaosShuffleIO(conf) + daosShuffleIO.initialize( + conf.getAllWithPrefix(ShuffleDataIOUtils.SHUFFLE_SPARK_CONF_PREFIX).toMap.asJava) + + // stop all executor threads when shutdown + ShutdownHookManager.addShutdownHook(() => daosShuffleIO.close()) + + val daosFinalizer = DaosClient.FINALIZER + + val finalizer = () => { + closeAllHadoopFs + daosFinalizer.run() + } + + if (io.daos.ShutdownHookManager.removeHook(daosFinalizer) || + org.apache.hadoop.util.ShutdownHookManager.get.removeShutdownHook(daosFinalizer)) { + ShutdownHookManager.addShutdownHook(finalizer) + logInfo("relocated daos finalizer") + } else { + logWarning("failed to relocate daos finalizer") + } + + /** + * A mapping from shuffle ids to the task ids of mappers producing output for those shuffles. + */ + private[this] val taskIdMapsForShuffle = new ConcurrentHashMap[Int, OpenHashSet[Long]]() + + /** + * register {@link ShuffleDependency} to pass to tasks. + * + * @param shuffleId + * unique ID of shuffle in job + * @param dependency + * shuffle dependency + * @tparam K + * type of KEY + * @tparam V + * type of VALUE + * @tparam C + * type of combined value + * @return {@link BaseShuffleHandle} + */ + override def registerShuffle[K, V, C]( + shuffleId: Int, + dependency: ShuffleDependency[K, V, C]): BaseShuffleHandle[K, V, C] + = { + new BaseShuffleHandle(shuffleId, dependency) + } + + override def getWriter[K, V]( + handle: ShuffleHandle, + mapId: Long, + context: TaskContext, + metrics: ShuffleWriteMetricsReporter): DaosShuffleWriter[K, V, _] + = { + val mapTaskIds = taskIdMapsForShuffle.computeIfAbsent( + handle.shuffleId, _ => new OpenHashSet[Long](16)) + mapTaskIds.synchronized { mapTaskIds.add(context.taskAttemptId()) } + new DaosShuffleWriter(handle.asInstanceOf[BaseShuffleHandle[K, V, _]], mapId, context, daosShuffleIO) + } + + override def getReader[K, C]( + handle: ShuffleHandle, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): DaosShuffleReader[K, C] + = { + val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId( + handle.shuffleId, startPartition, endPartition) + new DaosShuffleReader(handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, + metrics, daosShuffleIO, SparkEnv.get.serializerManager, + shouldBatchFetch = canUseBatchFetch(startPartition, endPartition, context)) + } + + override def getReaderForRange[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): DaosShuffleReader[K, C] + = { + val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByRange( + handle.shuffleId, startMapIndex, endMapIndex, startPartition, endPartition) + new DaosShuffleReader(handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, + metrics, daosShuffleIO, SparkEnv.get.serializerManager, + shouldBatchFetch = canUseBatchFetch(startPartition, endPartition, context)) + } + + override def unregisterShuffle(shuffleId: Int): Boolean = { + logInfo("unregistering shuffle: " + shuffleId) + taskIdMapsForShuffle.remove(shuffleId) + daosShuffleIO.removeShuffle(shuffleId) + } + + override def shuffleBlockResolver: ShuffleBlockResolver = null + + override def stop(): Unit = { + daosShuffleIO.close() + finalizer() + ShutdownHookManager.removeShutdownHook(finalizer) + logInfo("stopped " + classOf[DaosShuffleManager]) + } +} diff --git a/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/DaosShuffleReader.scala b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/DaosShuffleReader.scala new file mode 100644 index 00000000..24c6ba9c --- /dev/null +++ b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/DaosShuffleReader.scala @@ -0,0 +1,129 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos + +import org.apache.spark.{InterruptibleIterator, SparkEnv, TaskContext} +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.serializer.SerializerManager +import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleReader, ShuffleReadMetricsReporter} +import org.apache.spark.storage.{BlockId, BlockManagerId} +import org.apache.spark.util.CompletionIterator +import org.apache.spark.util.collection.ExternalSorter + +class DaosShuffleReader[K, C]( + handle: BaseShuffleHandle[K, _, C], + blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])], + context: TaskContext, + readMetrics: ShuffleReadMetricsReporter, + shuffleIO: DaosShuffleIO, + serializerManager: SerializerManager = SparkEnv.get.serializerManager, + shouldBatchFetch: Boolean = false) + extends ShuffleReader[K, C] with Logging { + + private val dep = handle.dependency + + private val conf = SparkEnv.get.conf + + private val daosReader = shuffleIO.getDaosReader(handle.shuffleId) + + override def read(): Iterator[Product2[K, C]] = { + val maxBytesInFlight = conf.get(SHUFFLE_DAOS_READ_MAX_BYTES_IN_FLIGHT) + val wrappedStreams = new ShufflePartitionIterator( + context, + blocksByAddress, + serializerManager.wrapStream, + maxBytesInFlight, + conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM), + conf.get(config.SHUFFLE_DETECT_CORRUPT), + conf.get(config.SHUFFLE_DETECT_CORRUPT_MEMORY), + readMetrics, + daosReader, + shouldBatchFetch + ).toCompletionIterator + + val serializerInstance = dep.serializer.newInstance() + + // Create a key/value iterator for each stream + val recordIter = wrappedStreams.flatMap { case (blockId, wrappedStream) => + // Note: the asKeyValueIterator below wraps a key/value iterator inside of a + // NextIterator. The NextIterator makes sure that close() is called on the + // underlying InputStream when all records have been read. + serializerInstance.deserializeStream(wrappedStream).asKeyValueIterator + } + + // Update the context task metrics for each record read. + val metricIter = CompletionIterator[(Any, Any), Iterator[(Any, Any)]]( + recordIter.map { record => + readMetrics.incRecordsRead(1) + record + }, + context.taskMetrics().mergeShuffleReadMetrics()) + + // An interruptible iterator must be used here in order to support task cancellation + val interruptibleIter = new InterruptibleIterator[(Any, Any)](context, metricIter) + + val aggregatedIter: Iterator[Product2[K, C]] = if (dep.aggregator.isDefined) { + if (dep.mapSideCombine) { + // We are reading values that are already combined + val combinedKeyValuesIterator = interruptibleIter.asInstanceOf[Iterator[(K, C)]] + dep.aggregator.get.combineCombinersByKey(combinedKeyValuesIterator, context) + } else { + // We don't know the value type, but also don't care -- the dependency *should* + // have made sure its compatible w/ this aggregator, which will convert the value + // type to the combined type C + val keyValuesIterator = interruptibleIter.asInstanceOf[Iterator[(K, Nothing)]] + dep.aggregator.get.combineValuesByKey(keyValuesIterator, context) + } + } else { + interruptibleIter.asInstanceOf[Iterator[Product2[K, C]]] + } + + // Sort the output if there is a sort ordering defined. + val resultIter = dep.keyOrdering match { + case Some(keyOrd: Ordering[K]) => + // Create an ExternalSorter to sort the data. + val sorter = + new ExternalSorter[K, C, C](context, ordering = Some(keyOrd), serializer = dep.serializer) + sorter.insertAll(aggregatedIter) + context.taskMetrics().incMemoryBytesSpilled(sorter.memoryBytesSpilled) + context.taskMetrics().incDiskBytesSpilled(sorter.diskBytesSpilled) + context.taskMetrics().incPeakExecutionMemory(sorter.peakMemoryUsedBytes) + // Use completion callback to stop sorter if task was finished/cancelled. + context.addTaskCompletionListener[Unit](_ => { + sorter.stop() + }) + CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]](sorter.iterator, sorter.stop()) + case None => + aggregatedIter + } + + resultIter match { + case _: InterruptibleIterator[Product2[K, C]] => resultIter + case _ => + // Use another interruptible iterator here to support task cancellation as aggregator + // or(and) sorter may have consumed previous interruptible iterator. + new InterruptibleIterator[Product2[K, C]](context, resultIter) + } + } +} diff --git a/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/DaosShuffleWriter.scala b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/DaosShuffleWriter.scala new file mode 100644 index 00000000..de283712 --- /dev/null +++ b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/DaosShuffleWriter.scala @@ -0,0 +1,101 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos + +import org.apache.spark.{SparkEnv, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.MapStatus +import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleWriter} + +class DaosShuffleWriter[K, V, C]( + handle: BaseShuffleHandle[K, V, C], + mapId: Long, + context: TaskContext, + shuffleIO: DaosShuffleIO) + extends ShuffleWriter[K, V] with Logging { + + private val dep = handle.dependency + + private var partitionsWriter: MapPartitionsWriter[K, V, _] = null + + private var stopping = false + + private var mapStatus: MapStatus = null + + private val blockManager = SparkEnv.get.blockManager + + override def write(records: Iterator[Product2[K, V]]): Unit = { +// val start = System.nanoTime() + partitionsWriter = if (dep.mapSideCombine) { + new MapPartitionsWriter[K, V, C]( + handle.shuffleId, + context, + dep.aggregator, + Some(dep.partitioner), + dep.keyOrdering, + dep.serializer, + shuffleIO) + } else { + // In this case we pass neither an aggregator nor an ordering to the sorter, because we don't + // care whether the keys get sorted in each partition; that will be done on the reduce side + // if the operation being run is sortByKey. + new MapPartitionsWriter[K, V, V]( + handle.shuffleId, + context, + aggregator = None, + Some(dep.partitioner), + ordering = None, + dep.serializer, + shuffleIO) + } + partitionsWriter.insertAll(records) + val partitionLengths = partitionsWriter.commitAll + + // logInfo(context.taskAttemptId() + " all time: " + (System.nanoTime() - start)/1000000) + + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapId) + } + + override def stop(success: Boolean): Option[MapStatus] = { + try { + if (stopping) { + return None + } + stopping = true + if (success) { + Option(mapStatus) + } else { + None + } + } finally { + // Clean up our sorter, which may have its own intermediate files + if (partitionsWriter != null) { + val startTime = System.nanoTime() + partitionsWriter.close + context.taskMetrics().shuffleWriteMetrics.incWriteTime(System.nanoTime - startTime) + partitionsWriter = null + } + } + } +} diff --git a/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/MapPartitionsWriter.scala b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/MapPartitionsWriter.scala new file mode 100644 index 00000000..d8ebb83e --- /dev/null +++ b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/MapPartitionsWriter.scala @@ -0,0 +1,494 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos + +import java.util.Comparator + +import org.apache.spark._ +import org.apache.spark.internal.Logging +import org.apache.spark.memory.{MemoryConsumer, TaskMemoryManager} +import org.apache.spark.serializer.Serializer +import org.apache.spark.util.collection.SizeTrackingAppendOnlyMap + +class MapPartitionsWriter[K, V, C]( + shuffleId: Int, + context: TaskContext, + aggregator: Option[Aggregator[K, V, C]] = None, + partitioner: Option[Partitioner] = None, + ordering: Option[Ordering[K]] = None, + serializer: Serializer = SparkEnv.get.serializer, + shuffleIO: DaosShuffleIO) extends Logging { + + private val conf = SparkEnv.get.conf + + private val numPartitions = partitioner.map(_.numPartitions).getOrElse(1) + private val shouldPartition = numPartitions > 1 + private def getPartition(key: K): Int = { + if (shouldPartition) partitioner.get.getPartition(key) else 0 + } + + private val serializerManager = SparkEnv.get.serializerManager + private val serInstance = serializer.newInstance() + + private val daosWriter = shuffleIO.getDaosWriter( + numPartitions, + shuffleId, + context.taskAttemptId()) + private val writeMetrics = context.taskMetrics().shuffleWriteMetrics + + /* key comparator if map-side combiner is defined */ + private val keyComparator: Comparator[K] = ordering.getOrElse((a: K, b: K) => { + val h1 = if (a == null) 0 else a.hashCode() + val h2 = if (b == null) 0 else b.hashCode() + if (h1 < h2) -1 else if (h1 == h2) 0 else 1 + }) + + private def comparator: Option[Comparator[K]] = { + if (ordering.isDefined || aggregator.isDefined) { + Some(keyComparator) + } else { + None + } + } + + // buffer by partition + @volatile var writeBuffer = new PartitionsBuffer[K, C]( + numPartitions, + comparator, + conf, + context.taskMemoryManager()) + + private[this] var _elementsRead = 0 + + private var _writtenBytes = 0L + def writtenBytes: Long = _writtenBytes + + def peakMemoryUsedBytes: Long = writeBuffer.peakSize + + def insertAll(records: Iterator[Product2[K, V]]): Unit = { + // TODO: stop combining if we find that the reduction factor isn't high + val start = System.nanoTime(); + val shouldCombine = aggregator.isDefined + if (shouldCombine) { + // Combine values in-memory first using our AppendOnlyMap + val mergeValue = aggregator.get.mergeValue + val createCombiner = aggregator.get.createCombiner + var kv: Product2[K, V] = null + val update = (hadValue: Boolean, oldValue: C) => { + if (hadValue) mergeValue(oldValue, kv._2) else createCombiner(kv._2) + } + while (records.hasNext) { + addElementsRead() + kv = records.next() + writeBuffer.changeValue(getPartition(kv._1), kv._1, update) + } + } else { + // Stick values into our buffer + while (records.hasNext) { + addElementsRead() + val kv = records.next() + writeBuffer.insert(getPartition(kv._1), kv._1, kv._2.asInstanceOf[C]) + } + } + // logInfo(context.taskAttemptId() + " insert time: " + (System.nanoTime() - start)/1000000) + } + + def commitAll: Array[Long] = { + writeBuffer.flushAll + writeBuffer.close + daosWriter.getPartitionLens(numPartitions) + } + + def close: Unit = { + // serialize rest of records + daosWriter.close + } + + protected def addElementsRead(): Unit = { _elementsRead += 1 } + + /** + * @param numPartitions + * @param keyComparator + * @param conf + * @param taskMemManager + * @tparam K + * @tparam C + */ + private[daos] class PartitionsBuffer[K, C]( + numPartitions: Int, + val keyComparator: Option[Comparator[K]], + val conf: SparkConf, + val taskMemManager: TaskMemoryManager) extends MemoryConsumer(taskMemManager) { + private val partBufferThreshold = conf.get(SHUFFLE_DAOS_WRITE_PARTITION_BUFFER_SIZE).toInt * 1024 + private val totalBufferThreshold = conf.get(SHUFFLE_DAOS_WRITE_BUFFER_SIZE).toInt * 1024 * 1024 + private val totalBufferInitial = conf.get(SHUFFLE_DAOS_WRITE_BUFFER_INITIAL_SIZE).toInt * 1024 * 1024 + private val forceWritePct = conf.get(SHUFFLE_DAOS_WRITE_BUFFER_FORCE_PCT) + private val totalWriteValve = totalBufferThreshold * forceWritePct + private[daos] val sampleStat = new SampleStat + + if (log.isDebugEnabled()) { + log.debug("partBufferThreshold: " + partBufferThreshold) + log.debug("totalBufferThreshold: " + totalBufferThreshold) + log.debug("totalBufferInitial: " + totalBufferInitial) + log.debug("forceWritePct: " + forceWritePct) + log.debug("totalWriteValve: " + totalWriteValve) + } + + if (totalBufferInitial > totalBufferThreshold) { + throw new IllegalArgumentException("total buffer initial size (" + totalBufferInitial + ") should be no more " + + "than total buffer threshold (" + totalBufferThreshold + ").") + } + + private var totalSize = 0L + private var memoryLimit = totalBufferInitial * 1L + private var largestSize = 0L + + var peakSize = 0L + + private def initialize[T >: Linked[K, C] with SizeAware[K, C]](): + (T, T, Array[SizeAwareMap[K, C]], Array[SizeAwareBuffer[K, C]]) = { + // create virtual partition head and end, as well as all linked partitions + val (partitionMapArray, partitionBufferArray) = if (comparator.isDefined) { + (new Array[SizeAwareMap[K, C]](numPartitions), null) + } else { + (null, new Array[SizeAwareBuffer[K, C]](numPartitions)) + } + val (head, end) = if (comparator.isDefined) { + val mapHead = new SizeAwareMap[K, C](-1, partBufferThreshold, + totalBufferInitial, taskMemManager, this) + val mapEnd = new SizeAwareMap[K, C](-2, partBufferThreshold, + totalBufferInitial, taskMemManager, this) + (0 until numPartitions).foreach(i => { + val map = new SizeAwareMap[K, C](i, partBufferThreshold, totalBufferInitial, taskMemManager, this) + partitionMapArray(i) = map + if (i > 0) { + val prevMap = partitionMapArray(i - 1) + prevMap.next = map + map.prev = prevMap + } + }) + (mapHead, mapEnd) + } else { + val bufferHead = new SizeAwareBuffer[K, C](-1, partBufferThreshold, + totalBufferInitial, taskMemManager, this) + val bufferEnd = new SizeAwareBuffer[K, C](-2, partBufferThreshold, + totalBufferInitial, taskMemManager, this) + (0 until numPartitions).foreach(i => { + val buffer = new SizeAwareBuffer[K, C](i, partBufferThreshold, totalBufferInitial, taskMemManager, this) + partitionBufferArray(i) = buffer + if (i > 0) { + val prevBuffer = partitionBufferArray(i - 1) + prevBuffer.next = buffer + buffer.prev = prevBuffer + } + }) + (bufferHead, bufferEnd) + } + // link head and end + val (first, last) = if (comparator.isDefined) (partitionMapArray(0), partitionMapArray(numPartitions - 1)) + else (partitionBufferArray(0), partitionBufferArray(numPartitions - 1)) + head.next = first + first.prev = head + end.prev = last + last.next = end + + (head, end, partitionMapArray, partitionBufferArray) + } + + private val (head, end, partitionMapArray, partitionBufferArray) = initialize() + + private def moveToFirst(node: Linked[K, C] with SizeAware[K, C]): Unit = { + if (head.next != node) { + // remove node from list + node.prev.next = node.next + node.next.prev = node.prev + // move to first + node.next = head.next + head.next.prev = node + head.next = node + node.prev = head + // set largestSize + largestSize = head.next.estimatedSize + } + } + + private def moveToLast(node: Linked[K, C] with SizeAware[K, C]): Unit = { + if (end.prev != node) { + // remove node from list + node.prev.next = node.next + node.next.prev = node.prev + // move to last + node.prev = end.prev + end.prev.next = node + end.prev = node + node.next = end + } + } + + def changeValue(partitionId: Int, key: K, updateFunc: (Boolean, C) => C): Unit = { + val map = partitionMapArray(partitionId) + val estSize = map.changeValue(key, updateFunc) + afterUpdate(estSize, map) + } + + def insert(partitionId: Int, key: K, value: C): Unit = { + val buffer = partitionBufferArray(partitionId) + val estSize = buffer.insert(key, value) + afterUpdate(estSize, buffer) + } + + def afterUpdate[T <: SizeAware[K, C] with Linked[K, C]](estSize: Long, buffer: T): Unit = { + if (estSize > largestSize) { + largestSize = estSize + moveToFirst(buffer) + } else if (estSize == 0) { + moveToLast(buffer) + } else { + // check if total buffer exceeds memory limit + maybeWriteTotal() + } + } + + private def writeFirst: Unit = { + val buffer = head.next + buffer.writeAndFlush + moveToLast(buffer) + } + + private def maybeWriteTotal(): Unit = { + if (totalSize > totalWriteValve) { + writeFirst + } + if (totalSize > memoryLimit) { + val memRequest = 2 * totalSize - memoryLimit + val granted = acquireMemory(memRequest) + memoryLimit += granted + if (totalSize >= memoryLimit) { + writeFirst + } + } + } + + def updateTotalSize(diff: Long): Unit = { + totalSize += diff + if (totalSize > peakSize) { + peakSize = totalSize + } + } + + def releaseMemory(memory: Long): Unit = { + memoryLimit -= memory + } + + def flushAll: Unit = { + val buffer = if (comparator.isDefined) partitionMapArray else partitionBufferArray + buffer.foreach(e => e.writeAndFlush) + } + + def close: Unit = { + val buffer = if (comparator.isDefined) partitionMapArray else partitionBufferArray + buffer.foreach(b => b.close) + } + + def spill(size: Long, trigger: MemoryConsumer): Long = ??? + } + + private[daos] trait SizeAware[K, C] { + this: MemoryConsumer => + + protected var writeCount = 0 + + protected var lastSize = 0L + + protected var _pairsWriter: PartitionOutput = null + + def partitionId: Int + + def writeThreshold: Int + + def estimatedSize: Long + + def totalBufferInitial: Long + + def iterator: Iterator[(K, C)] + + def reset: Unit + + def parent: PartitionsBuffer[K, C] + + def pairsWriter: PartitionOutput + + def updateTotalSize(estSize: Long): Unit = { + val diff = estSize - lastSize + if (diff > 0) { + lastSize = estSize + parent.updateTotalSize(diff) + } + } + + def releaseMemory(memory: Long): Unit = { + freeMemory(memory) + parent.releaseMemory(memory) + } + + private def writeAndFlush(memory: Long): Unit = { + val writer = if (_pairsWriter != null) _pairsWriter else pairsWriter + var count = 0 + iterator.foreach(p => { + writer.write(p._1, p._2) + count += 1 + }) + if (count > 0) { + writer.flush // force write + writeCount += count + lastSize = 0 + parent.updateTotalSize(-memory) + releaseMemory(memory - totalBufferInitial) + reset + } + } + + def writeAndFlush: Unit = { + writeAndFlush(estimatedSize) + } + + def maybeWrite(memory: Long): Boolean = { + if (memory < writeThreshold) { + false + } else { + writeAndFlush(memory) + true + } + } + + def afterUpdate(estSize: Long): Long = { + if (maybeWrite(estSize)) { + 0L + } else { + updateTotalSize(estSize) + estSize + } + } + + def close: Unit = { + if (_pairsWriter != null) { + _pairsWriter.close + _pairsWriter = null + } + } + } + + private[daos] trait Linked[K, C] { + this: SizeAware[K, C] => + + var prev: Linked[K, C] with SizeAware[K, C] = null + var next: Linked[K, C] with SizeAware[K, C] = null + } + + private class SizeAwareMap[K, C]( + val partitionId: Int, + val writeThreshold: Int, + val totalBufferInitial: Long, + taskMemoryManager: TaskMemoryManager, + val parent: PartitionsBuffer[K, C]) extends MemoryConsumer(taskMemoryManager) + with Linked[K, C] with SizeAware[K, C] { + + private var map = new SizeSamplerAppendOnlyMap[K, C](parent.sampleStat) + private var _estSize: Long = _ + + def estimatedSize: Long = _estSize + + def changeValue(key: K, updateFunc: (Boolean, C) => C): Long = { + map.changeValue(key, updateFunc) + _estSize = map.estimateSize() + afterUpdate(_estSize) + } + + def reset: Unit = { + map = new SizeSamplerAppendOnlyMap[K, C](parent.sampleStat) + _estSize = map.estimateSize() + } + + def iterator(): Iterator[(K, C)] = { + map.destructiveSortedIterator(parent.keyComparator.get) + } + + def spill(size: Long, trigger: MemoryConsumer): Long = { + val curSize = _estSize + writeAndFlush + curSize + } + + def pairsWriter: PartitionOutput = { + if (_pairsWriter == null) { + _pairsWriter = new PartitionOutput(shuffleId, context.taskAttemptId(), partitionId, serializerManager, + serInstance, daosWriter, writeMetrics) + } + _pairsWriter + } + } + + private class SizeAwareBuffer[K, C]( + val partitionId: Int, + val writeThreshold: Int, + val totalBufferInitial: Long, + taskMemoryManager: TaskMemoryManager, + val parent: PartitionsBuffer[K, C]) extends MemoryConsumer(taskMemoryManager) + with Linked[K, C] with SizeAware[K, C] { + + private var buffer = new SizeSamplerPairBuffer[K, C](parent.sampleStat) + private var _estSize: Long = _ + + def estimatedSize: Long = _estSize + + def insert(key: K, value: C): Long = { + buffer.insert(key, value) + _estSize = buffer.estimateSize() + afterUpdate(_estSize) + } + + def reset: Unit = { + buffer = new SizeSamplerPairBuffer[K, C](parent.sampleStat) + _estSize = buffer.estimateSize() + } + + def iterator(): Iterator[(K, C)] = { + buffer.iterator() + } + + def spill(size: Long, trigger: MemoryConsumer): Long = { + val curSize = _estSize + writeAndFlush + curSize + } + + def pairsWriter: PartitionOutput = { + if (_pairsWriter == null) { + _pairsWriter = new PartitionOutput(shuffleId, context.taskAttemptId(), partitionId, serializerManager, + serInstance, daosWriter, writeMetrics) + } + _pairsWriter + } + } +} diff --git a/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/PartitionOutput.scala b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/PartitionOutput.scala new file mode 100644 index 00000000..3626dcf2 --- /dev/null +++ b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/PartitionOutput.scala @@ -0,0 +1,117 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos + +import java.io.OutputStream + +import org.apache.spark.serializer.{SerializationStream, SerializerInstance, SerializerManager} +import org.apache.spark.shuffle.ShuffleWriteMetricsReporter +import org.apache.spark.storage.{ShuffleBlockId, TimeTrackingOutputStream} +import org.apache.spark.util.Utils + +/** + * Output for each partition. + * + * @param shuffleId + * @param mapId + * @param partitionId + * @param serializerManager + * @param serializerInstance + * @param daosWriter + * @param writeMetrics + */ +class PartitionOutput( + shuffleId: Int, + mapId: Long, + partitionId: Int, + serializerManager: SerializerManager, + serializerInstance: SerializerInstance, + daosWriter: DaosWriter, + writeMetrics: ShuffleWriteMetricsReporter) { + + private var ds: DaosShuffleOutputStream = null + + private var ts: TimeTrackingOutputStream = null + + private var bs: OutputStream = null + + private var objOut: SerializationStream = null + + private var opened: Boolean = false + + private var numRecordsWritten = 0 + + private var lastWrittenBytes = 0L + + def open: Unit = { + ds = new DaosShuffleOutputStream(partitionId, daosWriter) + ts = new TimeTrackingOutputStream(writeMetrics, ds) + bs = serializerManager.wrapStream(ShuffleBlockId(shuffleId, mapId, partitionId), ts) + objOut = serializerInstance.serializeStream(bs) + opened = true + } + + def write(key: Any, value: Any): Unit = { + if (!opened) { + open + } + objOut.writeKey(key) + objOut.writeValue(value) + // update metrics + numRecordsWritten += 1 + writeMetrics.incRecordsWritten(1) + if (numRecordsWritten % 16384 == 0) { + updateWrittenBytes + } + } + + private def updateWrittenBytes: Unit = { + val writtenBytes = ds.getWrittenBytes + writeMetrics.incBytesWritten(writtenBytes - lastWrittenBytes) + lastWrittenBytes = writtenBytes + } + + def flush: Unit = { + if (opened) { + objOut.flush() + daosWriter.flush(partitionId) + updateWrittenBytes + } + } + + def close: Unit = { + if (opened) { + Utils.tryWithSafeFinally { + objOut.close() + } { + updateWrittenBytes + objOut = null + bs = null + ds = null + ts = null + opened = false + } + } + } +} diff --git a/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/ShufflePartitionIterator.scala b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/ShufflePartitionIterator.scala new file mode 100644 index 00000000..f7727e86 --- /dev/null +++ b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/ShufflePartitionIterator.scala @@ -0,0 +1,243 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos + +import java.io.{InputStream, IOException} +import java.util + +import org.apache.commons.io.IOUtils + +import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.shuffle.{FetchFailedException, ShuffleReadMetricsReporter} +import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockBatchId, ShuffleBlockId} +import org.apache.spark.util.{CompletionIterator, TaskCompletionListener, Utils} + +class ShufflePartitionIterator( + context: TaskContext, + blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])], + streamWrapper: (BlockId, InputStream) => InputStream, + maxBytesInFlight: Long, + maxReqSizeShuffleToMem: Long, + detectCorrupt: Boolean, + detectCorruptUseExtraMemory: Boolean, + shuffleMetrics: ShuffleReadMetricsReporter, + daosReader: DaosReader, + doBatchFetch: Boolean) extends Iterator[(BlockId, InputStream)] with Logging { + + private var lastMapReduce: (java.lang.Long, Integer) = _ + private var lastMRBlock: (java.lang.Long, BlockId, BlockManagerId) = _ + + private[daos] var inputStream: DaosShuffleInputStream = null + + // (mapid, reduceid) -> (length, BlockId, BlockManagerId) + private val mapReduceIdMap = new util.LinkedHashMap[(java.lang.Long, Integer), + (java.lang.Long, BlockId, BlockManagerId)] + + private var mapReduceIt: util.Iterator[util.Map.Entry[(java.lang.Long, Integer), + (java.lang.Long, BlockId, BlockManagerId)]] = _ + + private val onCompleteCallback = new ShufflePartitionCompletionListener(this) + + initialize + + def initialize: Unit = { + context.addTaskCompletionListener(onCompleteCallback) + startReading + } + + private def getMapReduceId(shuffleBlockId: ShuffleBlockId): (java.lang.Long, Integer) = { + val name = shuffleBlockId.name.split("_") + (java.lang.Long.valueOf(name(2)), Integer.valueOf(name(3))) + } + + private def startReading: Unit = { + blocksByAddress.foreach(t2 => { + t2._2.foreach(t3 => { + val mapReduceId = getMapReduceId(t3._1.asInstanceOf[ShuffleBlockId]) + if (mapReduceIdMap.containsKey(mapReduceId._1)) { + throw new IllegalStateException("duplicate map id: " + mapReduceId._1) + } + mapReduceIdMap.put((mapReduceId._1, mapReduceId._2), (t3._2, t3._1, t2._1)) + }) + }) + + if (log.isDebugEnabled) { + log.debug(s"total mapreduceId: ${mapReduceIdMap.size()}, they are, ") + mapReduceIdMap.forEach((key, value) => logDebug(key.toString() + " = " + value.toString)) + } + + inputStream = new DaosShuffleInputStream(daosReader, mapReduceIdMap, + maxBytesInFlight, maxReqSizeShuffleToMem, shuffleMetrics) + mapReduceIt = mapReduceIdMap.entrySet().iterator() + } + + override def hasNext: Boolean = { + (!inputStream.isCompleted()) && mapReduceIt.hasNext + } + + override def next(): (BlockId, InputStream) = { + if (!hasNext) { + throw new NoSuchElementException + } + val entry = mapReduceIt.next() + lastMapReduce = entry.getKey + lastMRBlock = entry.getValue + val lastBlockId = lastMRBlock._2.asInstanceOf[ShuffleBlockId] + inputStream.nextMap() + var input: InputStream = null + var streamCompressedOrEncryptd = false + try { + input = streamWrapper(lastBlockId, inputStream) + streamCompressedOrEncryptd = !input.eq(inputStream) + if (streamCompressedOrEncryptd && detectCorruptUseExtraMemory) { + input = Utils.copyStreamUpTo(input, maxBytesInFlight / 3) + } + } catch { + case e: IOException => + logError(s"got an corrupted block ${inputStream.getCurBlockId} originated from " + + s"${inputStream.getCurOriginAddress}.", e) + throw e + } finally { + if (input == null) { + inputStream.close(false) + } + } + (lastBlockId, new BufferReleasingInputStream(lastMapReduce, lastMRBlock, input, this, + detectCorrupt && streamCompressedOrEncryptd)) + } + + def throwFetchFailedException( + blockId: BlockId, + mapIndex: Int, + address: BlockManagerId, + e: Throwable): Nothing = { + blockId match { + case ShuffleBlockId(shufId, mapId, reduceId) => + throw new FetchFailedException(address, shufId, mapId, mapIndex, reduceId, e) + case ShuffleBlockBatchId(shuffleId, mapId, startReduceId, _) => + throw new FetchFailedException(address, shuffleId, mapId, mapIndex, startReduceId, e) + case _ => + throw new SparkException( + "Failed to get block " + blockId + ", which is not a shuffle block", e) + } + } + + def toCompletionIterator: Iterator[(BlockId, InputStream)] = { + CompletionIterator[(BlockId, InputStream), this.type](this, + onCompleteCallback.onTaskCompletion(context)) + } + + def cleanup: Unit = { + if (inputStream != null) { + inputStream.close(false) + inputStream = null; + } + } + +} + +/** + * Helper class that ensures a ManagedBuffer is released upon InputStream.close() and + * also detects stream corruption if streamCompressedOrEncrypted is true + */ +private class BufferReleasingInputStream( + // This is visible for testing + private val mapreduce: (java.lang.Long, Integer), + private val mrblock: (java.lang.Long, BlockId, BlockManagerId), + private val delegate: InputStream, + private val iterator: ShufflePartitionIterator, + private val detectCorruption: Boolean) + extends InputStream { + private[this] var closed = false + + override def read(): Int = { + try { + delegate.read() + } catch { + case e: IOException if detectCorruption => + IOUtils.closeQuietly(this) + iterator.throwFetchFailedException(mrblock._2, mapreduce._1.toInt, + mrblock._3, e) + } + } + + override def close(): Unit = { + if (!closed) { + delegate.close() + closed = true + } + } + + override def available(): Int = delegate.available() + + override def mark(readlimit: Int): Unit = delegate.mark(readlimit) + + override def skip(n: Long): Long = { + try { + delegate.skip(n) + } catch { + case e: IOException if detectCorruption => + IOUtils.closeQuietly(this) + iterator.throwFetchFailedException(mrblock._2, mapreduce._1.toInt, + mrblock._3, e) + } + } + + override def markSupported(): Boolean = delegate.markSupported() + + override def read(b: Array[Byte]): Int = { + try { + delegate.read(b) + } catch { + case e: IOException if detectCorruption => + IOUtils.closeQuietly(this) + iterator.throwFetchFailedException(mrblock._2, mapreduce._1.toInt, + mrblock._3, e) + } + } + + override def read(b: Array[Byte], off: Int, len: Int): Int = { + try { + delegate.read(b, off, len) + } catch { + case e: IOException if detectCorruption => + IOUtils.closeQuietly(this) + iterator.throwFetchFailedException(mrblock._2, mapreduce._1.toInt, + mrblock._3, e) + } + } + + override def reset(): Unit = delegate.reset() +} + +private class ShufflePartitionCompletionListener(var data: ShufflePartitionIterator) + extends TaskCompletionListener { + + override def onTaskCompletion(context: TaskContext): Unit = { + if (data != null) { + data.cleanup + } + } +} diff --git a/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/SizeSampler.scala b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/SizeSampler.scala new file mode 100644 index 00000000..faf4dbf0 --- /dev/null +++ b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/SizeSampler.scala @@ -0,0 +1,138 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos + +import scala.collection.mutable + +import org.apache.spark.util.SizeEstimator + +/** + * A trait to sample size of object. It mimics what {@link SizeTracker} does. The differences between them are, + * - this trait is for sampling size of each partition buffer. + * - this trait lets caller control when to sample size. + * - bytesPerUpdate is calculated and shared among all buffers. + */ +private[spark] trait SizeSampler { + + import SizeSampler._ + + /** Samples taken since last resetSamples(). Only the last two are kept for extrapolation. */ + private val samples = new mutable.Queue[Sample] + + /** Total number of insertions and updates into the map since the last resetSamples(). */ + private var numUpdates: Long = _ + + private var bytesPerUpdate: Double = _ + + private var stat: SampleStat = _ + + protected def setSampleStat(stat: SampleStat): Unit = { + this.stat = stat + } + + /** + * Reset samples collected so far. + * This should be called after the collection undergoes a dramatic change in size. + */ + protected def resetSamples(): Unit = { + numUpdates = 1 + samples.clear() + takeSample() + } + + protected def afterUpdate(): Unit = { + numUpdates += 1 + stat.incNumUpdates + if (stat.needSample) { + takeSample() + } + } + + /** + * Take a new sample of the current collection's size. + */ + protected def takeSample(): Unit = { + samples.enqueue(Sample(SizeEstimator.estimate(this), numUpdates)) + // Only use the last two samples to extrapolate + if (samples.size > 2) { + samples.dequeue() + } + var updateDelta = 0L + val bytesDelta = samples.toList.reverse match { + case latest :: previous :: _ => + updateDelta = latest.numUpdates - previous.numUpdates + (latest.size - previous.size).toDouble / updateDelta + // If fewer than 2 samples, assume no change + case _ => 0 + } + if (updateDelta == 0) { + return + } + bytesPerUpdate = math.max(0, bytesDelta) + stat.updateStat(bytesPerUpdate, updateDelta) + } + + /** + * Estimate the current size of the collection in bytes. O(1) time. + */ + def estimateSize(): Long = { + assert(samples.nonEmpty) + val bpu = if (bytesPerUpdate == 0) stat.bytesPerUpdate else bytesPerUpdate + val extrapolatedDelta = bpu * (numUpdates - samples.last.numUpdates) + (samples.last.size + extrapolatedDelta).toLong + } +} + +private[spark] class SampleStat { + /** + * Controls the base of the exponential which governs the rate of sampling. + * E.g., a value of 2 would mean we sample at 1, 2, 4, 8, ... elements. + */ + private val SAMPLE_GROWTH_RATE = 1.1 + + private[daos] var numUpdates: Long = 0 + private[daos] var lastNumUpdates: Long = 0 + private[daos] var nextSampleNum: Long = 1 + private[daos] var bytesPerUpdate: Double = 0 + + def updateStat(partBpu: Double, partUpdateDelta: Long): Unit = { + bytesPerUpdate = ((numUpdates - partUpdateDelta) * bytesPerUpdate + + partUpdateDelta * partBpu + ) / numUpdates + lastNumUpdates = numUpdates + nextSampleNum = math.ceil(numUpdates * SAMPLE_GROWTH_RATE).toLong + } + + def needSample: Boolean = { + numUpdates == nextSampleNum + } + + def incNumUpdates: Unit = { + numUpdates += 1 + } +} + +private object SizeSampler { + case class Sample(size: Long, numUpdates: Long) +} diff --git a/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/SizeSamplerAppendOnlyMap.scala b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/SizeSamplerAppendOnlyMap.scala new file mode 100644 index 00000000..4ef1645a --- /dev/null +++ b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/SizeSamplerAppendOnlyMap.scala @@ -0,0 +1,50 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos + +import org.apache.spark.util.collection.AppendOnlyMap + +private[spark] class SizeSamplerAppendOnlyMap[K, V](val stat: SampleStat) + extends AppendOnlyMap[K, V] with SizeSampler +{ + + setSampleStat(stat) + resetSamples() + + override def update(key: K, value: V): Unit = { + super.update(key, value) + super.afterUpdate() + } + + override def changeValue(key: K, updateFunc: (Boolean, V) => V): V = { + val newValue = super.changeValue(key, updateFunc) + super.afterUpdate() + newValue + } + + override protected def growTable(): Unit = { + super.growTable() + resetSamples() + } +} diff --git a/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/SizeSamplerPairBuffer.scala b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/SizeSamplerPairBuffer.scala new file mode 100644 index 00000000..bde5d969 --- /dev/null +++ b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/SizeSamplerPairBuffer.scala @@ -0,0 +1,100 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos + +import org.apache.spark.unsafe.array.ByteArrayMethods +import org.apache.spark.util.collection.SizeTracker + +/** + * Pair buffer for each partition. + * + * @param stat + * @param initialCapacity + * @tparam K + * @tparam V + */ +class SizeSamplerPairBuffer[K, V](val stat: SampleStat, initialCapacity: Int = 64) extends SizeSampler { + import SizeSamplerPairBuffer._ + + require(initialCapacity <= MAXIMUM_CAPACITY, + s"Can't make capacity bigger than ${MAXIMUM_CAPACITY} elements") + require(initialCapacity >= 1, "Invalid initial capacity") + + // Basic growable array data structure. We use a single array of AnyRef to hold both the keys + // and the values, so that we can sort them efficiently with KVArraySortDataFormat. + private var capacity = initialCapacity + private var curSize = 0 + private var data = new Array[AnyRef](2 * initialCapacity) + + setSampleStat(stat) + resetSamples() + + /** Add an element into the buffer */ + def insert(key: K, value: V): Unit = { + if (curSize == capacity) { + growArray() + } + data(2 * curSize) = key.asInstanceOf[AnyRef] + data(2 * curSize + 1) = value.asInstanceOf[AnyRef] + curSize += 1 + afterUpdate() + } + + /** Double the size of the array because we've reached capacity */ + protected def growArray(): Unit = { + if (capacity >= MAXIMUM_CAPACITY) { + throw new IllegalStateException(s"Can't insert more than ${MAXIMUM_CAPACITY} elements") + } + val newCapacity = + if (capacity * 2 > MAXIMUM_CAPACITY) { // Overflow + MAXIMUM_CAPACITY + } else { + capacity * 2 + } + val newArray = new Array[AnyRef](2 * newCapacity) + System.arraycopy(data, 0, newArray, 0, 2 * capacity) + data = newArray + capacity = newCapacity + resetSamples() + } + + def iterator(): Iterator[(K, V)] = new Iterator[(K, V)] { + var pos = 0 + + override def hasNext: Boolean = pos < curSize + + override def next(): (K, V) = { + if (!hasNext) { + throw new NoSuchElementException + } + val pair = (data(2 * pos).asInstanceOf[K], data(2 * pos + 1).asInstanceOf[V]) + pos += 1 + pair + } + } +} + +private object SizeSamplerPairBuffer { + val MAXIMUM_CAPACITY: Int = ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 2 +} diff --git a/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/package.scala b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/package.scala new file mode 100644 index 00000000..f38f68a9 --- /dev/null +++ b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/package.scala @@ -0,0 +1,233 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle + +import org.apache.spark.internal.config.ConfigBuilder +import org.apache.spark.network.util.ByteUnit + +package object daos { + + val SHUFFLE_DAOS_APP_ID = "spark.shuffle.daos.app.id" + + val SHUFFLE_DAOS_POOL_UUID = + ConfigBuilder("spark.shuffle.daos.pool.uuid") + .version("3.0.0") + .stringConf + .createWithDefault(null) + + val SHUFFLE_DAOS_CONTAINER_UUID = + ConfigBuilder("spark.shuffle.daos.container.uuid") + .version("3.0.0") + .stringConf + .createWithDefault(null) + + val SHUFFLE_DAOS_POOL_RANKS = + ConfigBuilder("spark.shuffle.daos.ranks") + .version("3.0.0") + .stringConf + .createWithDefault("0") + + val SHUFFLE_DAOS_REMOVE_SHUFFLE_DATA = + ConfigBuilder("spark.shuffle.remove.shuffle.data") + .doc("remove shuffle data from DAOS after shuffle completed. Default is true") + .version("3.0.0") + .booleanConf + .createWithDefault(true) + + val SHUFFLE_DAOS_WRITE_PARTITION_BUFFER_SIZE = + ConfigBuilder("spark.shuffle.daos.write.partition.buffer") + .doc("size of the in-memory buffer for each map partition output, in KiB") + .version("3.0.0") + .bytesConf(ByteUnit.KiB) + .checkValue(v => v > 0, + s"The map partition buffer size must be positive.") + .createWithDefaultString("2048k") + + val SHUFFLE_DAOS_WRITE_BUFFER_SIZE = + ConfigBuilder("spark.shuffle.daos.write.buffer") + .doc("total size of in-memory buffers of each map's all partitions, in MiB") + .version("3.0.0") + .bytesConf(ByteUnit.MiB) + .checkValue(v => v > 50, + s"The total buffer size must be bigger than 50m.") + .createWithDefaultString("800m") + + val SHUFFLE_DAOS_WRITE_BUFFER_INITIAL_SIZE = + ConfigBuilder("spark.shuffle.daos.write.buffer.initial") + .doc("initial size of total in-memory buffer for each map output, in MiB") + .version("3.0.0") + .bytesConf(ByteUnit.MiB) + .checkValue(v => v > 10, + s"The initial total buffer size must be bigger than 10m.") + .createWithDefaultString("80m") + + val SHUFFLE_DAOS_WRITE_BUFFER_FORCE_PCT = + ConfigBuilder("spark.shuffle.daos.write.buffer.percentage") + .doc("percentage of spark.shuffle.daos.buffer. Force write some buffer data out when size is bigger than " + + "spark.shuffle.daos.buffer * (this percentage)") + .version("3.0.0") + .doubleConf + .checkValue(v => v >= 0.5 && v <= 0.9, + s"The percentage must be no less than 0.5 and less than or equal to 0.9") + .createWithDefault(0.75) + + val SHUFFLE_DAOS_WRITE_MINIMUM_SIZE = + ConfigBuilder("spark.shuffle.daos.write.minimum") + .doc("minimum size when write to DAOS, in KiB. A warning will be generated when size is less than this value" + + " and spark.shuffle.daos.write.warn.small is set to true") + .version("3.0.0") + .bytesConf(ByteUnit.KiB) + .checkValue(v => v > 0, + s"The DAOS write minimum size must be positive") + .createWithDefaultString("128k") + + val SHUFFLE_DAOS_WRITE_WARN_SMALL_SIZE = + ConfigBuilder("spark.shuffle.daos.write.warn.small") + .doc("log warning message when the size of written data is smaller than spark.shuffle.daos.write.minimum." + + " Default is false") + .version("3.0.0") + .booleanConf + .createWithDefault(false) + + val SHUFFLE_DAOS_WRITE_SINGLE_BUFFER_SIZE = + ConfigBuilder("spark.shuffle.daos.write.buffer.single") + .doc("size of single buffer for holding data to be written to DAOS") + .version("3.0.0") + .bytesConf(ByteUnit.MiB) + .checkValue(v => v >= 1, + s"The single DAOS write buffer must be at least 1m") + .createWithDefaultString("2m") + + val SHUFFLE_DAOS_READ_MINIMUM_SIZE = + ConfigBuilder("spark.shuffle.daos.read.minimum") + .doc("minimum size when read from DAOS, in KiB. ") + .version("3.0.0") + .bytesConf(ByteUnit.KiB) + .checkValue(v => v > 0, + s"The DAOS read minimum size must be positive") + .createWithDefaultString("2048k") + + val SHUFFLE_DAOS_READ_MAX_BYTES_IN_FLIGHT = + ConfigBuilder("spark.shuffle.daos.read.maxbytes.inflight") + .doc("maximum size of requested data when read from DAOS, in KiB. ") + .version("3.0.0") + .bytesConf(ByteUnit.KiB) + .checkValue(v => v > 0, + s"The DAOS read max bytes in flight must be positive") + .createWithDefaultString("10240k") + + val SHUFFLE_DAOS_WRITE_MAX_BYTES_IN_FLIGHT = + ConfigBuilder("spark.shuffle.daos.write.maxbytes.inflight") + .doc("maximum size of requested data when write to DAOS, in KiB. ") + .version("3.0.0") + .bytesConf(ByteUnit.KiB) + .checkValue(v => v > 0, + s"The DAOS write max bytes in flight must be positive") + .createWithDefaultString("20480k") + + val SHUFFLE_DAOS_READ_THREADS = + ConfigBuilder("spark.shuffle.daos.read.threads") + .doc("number of threads for each executor to read shuffle data concurrently. -1 means use number of executor " + + "cores.") + .version("3.0.0") + .intConf + .createWithDefault(1) + + val SHUFFLE_DAOS_WRITE_THREADS = + ConfigBuilder("spark.shuffle.daos.write.threads") + .doc("number of threads for each executor to write shuffle data concurrently. -1 means use number of executor " + + "cores.") + .version("3.0.0") + .intConf + .createWithDefault(1) + + val SHUFFLE_DAOS_READ_BATCH_SIZE = + ConfigBuilder("spark.shuffle.daos.read.batch") + .doc("number of read tasks to submit at most at each time") + .version("3.0.0") + .intConf + .checkValue(v => v > 0, + s"read batch size must be positive") + .createWithDefault(10) + + val SHUFFLE_DAOS_WRITE_SUBMITTED_LIMIT = + ConfigBuilder("spark.shuffle.daos.write.submitted.limit") + .doc("limit of number of write tasks to submit") + .version("3.0.0") + .intConf + .checkValue(v => v > 0, + s"limit of submitted task must be positive") + .createWithDefault(20) + + val SHUFFLE_DAOS_READ_WAIT_DATA_MS = + ConfigBuilder("spark.shuffle.daos.read.waitdata.ms") + .doc("number of milliseconds to wait data being read from other thread before timed out") + .version("3.0.0") + .intConf + .checkValue(v => v > 0, + s"wait data time must be positive") + .createWithDefault(100) + + val SHUFFLE_DAOS_WRITE_WAIT_MS = + ConfigBuilder("spark.shuffle.daos.write.waitdata.ms") + .doc("number of milliseconds to wait data being written in other thread before timed out") + .version("3.0.0") + .intConf + .checkValue(v => v > 0, + s"wait data time must be positive") + .createWithDefault(100) + + val SHUFFLE_DAOS_READ_WAIT_DATA_TIMEOUT_TIMES = + ConfigBuilder("spark.shuffle.daos.read.wait.timeout.times") + .doc("number of wait timeout (spark.shuffle.daos.read.waitdata.ms) after which shuffle read task reads data " + + "by itself instead of dedicated read thread") + .version("3.0.0") + .intConf + .checkValue(v => v > 0, + s"wait data timeout times must be positive") + .createWithDefault(5) + + val SHUFFLE_DAOS_WRITE_WAIT_DATA_TIMEOUT_TIMES = + ConfigBuilder("spark.shuffle.daos.write.wait.timeout.times") + .doc("number of wait timeout (spark.shuffle.daos.write.waitdata.ms) after which shuffle write task fails") + .version("3.0.0") + .intConf + .checkValue(v => v > 0, + s"wait data timeout times must be positive") + .createWithDefault(10) + + val SHUFFLE_DAOS_READ_FROM_OTHER_THREAD = + ConfigBuilder("spark.shuffle.daos.read.from.other.threads") + .doc("whether read shuffled data from other threads or not. true by default") + .version("3.0.0") + .booleanConf + .createWithDefault(true) + + val SHUFFLE_DAOS_WRITE_IN_OTHER_THREAD = + ConfigBuilder("spark.shuffle.daos.write.in.other.threads") + .doc("whether write shuffled data in other threads or not. true by default") + .version("3.0.0") + .booleanConf + .createWithDefault(true) +} diff --git a/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleIOTest.java b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleIOTest.java new file mode 100644 index 00000000..11bbdc6e --- /dev/null +++ b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleIOTest.java @@ -0,0 +1,106 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos; + +import io.daos.obj.DaosObjClient; +import io.daos.obj.DaosObject; +import io.daos.obj.DaosObjectId; +import org.apache.spark.SparkConf; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.core.classloader.annotations.SuppressStaticInitializationFor; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.lang.reflect.Field; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +@RunWith(PowerMockRunner.class) +@PowerMockIgnore("javax.management.*") +@PrepareForTest(DaosShuffleIO.class) +@SuppressStaticInitializationFor("io.daos.obj.DaosObjClient") +public class DaosShuffleIOTest { + + @Test + public void testSingleObjectInstanceOpen() throws Exception { + SparkConf testConf = new SparkConf(false); + testConf.set(package$.MODULE$.SHUFFLE_DAOS_READ_FROM_OTHER_THREAD(), false); + long appId = 1234567; + int shuffleId = 1; + testConf.set("spark.app.id", String.valueOf(appId)); + Field clientField = DaosShuffleIO.class.getDeclaredField("objClient"); + clientField.setAccessible(true); + DaosShuffleIO io = new DaosShuffleIO(testConf); + + DaosObjectId id = PowerMockito.mock(DaosObjectId.class); + PowerMockito.whenNew(DaosObjectId.class).withArguments(appId, Long.valueOf(shuffleId)).thenReturn(id); + Mockito.doNothing().when(id).encode(); + Mockito.when(id.isEncoded()).thenReturn(true); + DaosObject daosObject = PowerMockito.mock(DaosObject.class); + DaosObjClient client = PowerMockito.mock(DaosObjClient.class); + Mockito.when(client.getObject(id)).thenReturn(daosObject); + + AtomicBoolean open = new AtomicBoolean(false); + Mockito.when(daosObject.isOpen()).then(invocationOnMock -> + open.get() + ); + Mockito.doAnswer(invocationOnMock -> { + open.compareAndSet(false, true); + return invocationOnMock; + }).when(daosObject).open(); + clientField.set(io, client); + + int numThreads = 50; + ExecutorService es = Executors.newFixedThreadPool(numThreads); + AtomicInteger count = new AtomicInteger(0); + + Runnable r = () -> { + try { + DaosReader reader = io.getDaosReader(shuffleId); + if (reader.getObject() == daosObject && reader.getObject().isOpen()) { + count.incrementAndGet(); + } + } catch (Exception e) { + e.printStackTrace(); + } + }; + + for (int i = 0; i < numThreads; i++) { + es.submit(r); + } + + es.shutdown(); + es.awaitTermination(5, TimeUnit.SECONDS); + + Assert.assertEquals(50, count.intValue()); + } +} diff --git a/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleInputStreamTest.java b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleInputStreamTest.java new file mode 100644 index 00000000..8b47c6d8 --- /dev/null +++ b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleInputStreamTest.java @@ -0,0 +1,394 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos; + +import io.daos.obj.DaosObjClient; +import io.daos.obj.DaosObject; +import io.daos.obj.DaosObjectId; +import io.daos.obj.IODataDesc; +import io.netty.buffer.ByteBuf; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; +import org.apache.spark.TaskContext; +import org.apache.spark.shuffle.ShuffleReadMetricsReporter; +import org.apache.spark.storage.BlockId; +import org.apache.spark.storage.BlockManagerId; +import org.apache.spark.storage.ShuffleBlockId; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.SuppressStaticInitializationFor; +import org.powermock.modules.junit4.PowerMockRunner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Option; +import scala.Tuple2; +import scala.Tuple3; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Method; +import java.util.*; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.mockito.ArgumentMatchers.any; + +@RunWith(PowerMockRunner.class) +@PowerMockIgnore("javax.management.*") +@SuppressStaticInitializationFor("io.daos.obj.DaosObjClient") +public class DaosShuffleInputStreamTest { + + private static final Logger LOG = LoggerFactory.getLogger(DaosShuffleInputStreamTest.class); + + @Test + public void testReadFromOtherThreadCancelMultipleTimes1() throws Exception { + Map maps = new HashMap<>(); + maps.put("4", new AtomicInteger(0)); + maps.put("7", new AtomicInteger(0)); + maps.put("30", new AtomicInteger(0)); + readFromOtherThreadCancelMultipleTimes(maps); + } + + @Test + public void testReadFromOtherThreadCancelMultipleTimes2() throws Exception { + Map maps = new HashMap<>(); + maps.put("0", new AtomicInteger(0)); + maps.put("7", new AtomicInteger(0)); + maps.put("30", new AtomicInteger(0)); + readFromOtherThreadCancelMultipleTimes(maps); + } + + @Test + public void testReadFromOtherThreadCancelMultipleTimes3() throws Exception { + Map maps = new HashMap<>(); + maps.put("4", new AtomicInteger(0)); + maps.put("7", new AtomicInteger(0)); + maps.put("40", new AtomicInteger(0)); + readFromOtherThreadCancelMultipleTimes(maps); + } + + @Test + public void testReadFromOtherThreadCancelMultipleTimes4() throws Exception { + Map maps = new HashMap<>(); + maps.put("0", new AtomicInteger(0)); + maps.put("7", new AtomicInteger(0)); + maps.put("40", new AtomicInteger(0)); + readFromOtherThreadCancelMultipleTimes(maps); + } + + @Test + public void testReadFromOtherThreadCancelMultipleTimesConsecutiveLast() throws Exception { + Map maps = new HashMap<>(); + maps.put("0", new AtomicInteger(0)); + maps.put("39", new AtomicInteger(0)); + maps.put("40", new AtomicInteger(0)); + readFromOtherThreadCancelMultipleTimes(maps); + } + + @Test + public void testReadFromOtherThreadCancelMultipleTimesConsecutiveEarlier() throws Exception { + Map maps = new HashMap<>(); + maps.put("2", new AtomicInteger(0)); + maps.put("4", new AtomicInteger(0)); + maps.put("6", new AtomicInteger(0)); + maps.put("8", new AtomicInteger(0)); + readFromOtherThreadCancelMultipleTimes(maps); + } + + @Test + public void testReadFromOtherThreadCancelMultipleTimesLongWait() throws Exception { + Map maps = new HashMap<>(); + maps.put("2", new AtomicInteger(0)); + maps.put("4", new AtomicInteger(0)); + readFromOtherThreadCancelMultipleTimes(maps, 400); + } + + @Test + public void testReadFromOtherThreadCancelAll() throws Exception { + Map maps = new HashMap<>(); + maps.put("4", new AtomicInteger(0)); + maps.put("7", new AtomicInteger(0)); + maps.put("12", new AtomicInteger(0)); + maps.put("24", new AtomicInteger(0)); + maps.put("30", new AtomicInteger(0)); + maps.put("40", new AtomicInteger(0)); + readFromOtherThreadCancelMultipleTimes(maps, 500, false); + } + + public void readFromOtherThreadCancelMultipleTimes(Map maps) throws Exception { + readFromOtherThreadCancelMultipleTimes(maps, 500); + } + + public void readFromOtherThreadCancelMultipleTimes(Map maps, + int addWaitTimeMs) throws Exception { + readFromOtherThreadCancelMultipleTimes(maps, addWaitTimeMs, true); + } + + public void readFromOtherThreadCancelMultipleTimes(Map maps, + int addWaitTimeMs, boolean fromOtherThread) throws Exception { + int waitDataTimeMs = (int)new SparkConf(false).get(package$.MODULE$.SHUFFLE_DAOS_READ_WAIT_DATA_MS()); + int expectedFetchTimes = 32; + AtomicInteger fetchTimes = new AtomicInteger(0); + boolean[] succeeded = new boolean[] {true}; + Method method = IODataDesc.class.getDeclaredMethod("succeed"); + method.setAccessible(true); + CountDownLatch latch = new CountDownLatch(expectedFetchTimes); + + Thread callerThread = Thread.currentThread(); + + Answer answer = (invocationOnMock -> + { + fetchTimes.getAndIncrement(); + IODataDesc desc = invocationOnMock.getArgument(0); + desc.encode(); + method.invoke(desc); + + IODataDesc.Entry entry = desc.getEntry(0); + String mapId = entry.getKey(); + if (maps.containsKey(mapId)) { +// Thread thread = maps.get(mapId); + if (callerThread != Thread.currentThread()) { +// wait.incrementAndGet(); + // sleep to cause read timeout + System.out.println("sleeping at " + mapId); + Thread.sleep(waitDataTimeMs + addWaitTimeMs); + if ("40".equals(mapId) && !fromOtherThread) { // return data finally in case fromOtherThread is disabled + setLength(desc, succeeded, null); + latch.countDown(); + } + System.out.println("slept at " + mapId); + + return invocationOnMock; + } else { + latch.countDown(); + setLength(desc, succeeded, null); + System.out.println("read again by self at " + mapId); + return invocationOnMock; + } + } + latch.countDown(); + setLength(desc, succeeded, null); + return invocationOnMock; + }); + read(42, answer, latch, fetchTimes, succeeded); + } + + @Test + public void testReadFromOtherThreadCancelOnceAtLast() throws Exception { + testReadFromOtherThreadCancelOnce(40, 10240, 500); + } + + @Test + public void testReadFromOtherThreadCancelOnceAtMiddle() throws Exception { + testReadFromOtherThreadCancelOnce(2, 0, 500); + } + + @Test + public void testReadFromOtherThreadCancelOnceAtFirst() throws Exception { + testReadFromOtherThreadCancelOnce(0, 10*1024, 500); + } + + private void testReadFromOtherThreadCancelOnce(int pos, int desiredOffset, int addWaitMs) throws Exception { + int waitDataTimeMs = (int)new SparkConf(false).get(package$.MODULE$.SHUFFLE_DAOS_READ_WAIT_DATA_MS()); + int expectedFetchTimes = 32; + AtomicInteger fetchTimes = new AtomicInteger(0); + boolean[] succeeded = new boolean[] {true}; + AtomicInteger wait = new AtomicInteger(0); + Method method = IODataDesc.class.getDeclaredMethod("succeed"); + method.setAccessible(true); + CountDownLatch latch = new CountDownLatch(expectedFetchTimes); + + Answer answer = (invocationOnMock -> + { + fetchTimes.getAndIncrement(); + IODataDesc desc = invocationOnMock.getArgument(0); + desc.encode(); + method.invoke(desc); + + IODataDesc.Entry entry = desc.getEntry(0); + int offset = entry.getOffset(); + if (String.valueOf(pos).equals(entry.getKey()) && offset == desiredOffset) { + if (wait.get() == 0) { + wait.incrementAndGet(); + Thread.sleep(waitDataTimeMs + addWaitMs); + System.out.println("sleep at " + pos); + return invocationOnMock; + } else { + latch.countDown(); + checkAndSetSize(desc, succeeded, (15 * 1024 - desiredOffset), (5 * 1024) + desiredOffset); + System.out.println("self read later at " + pos); + return invocationOnMock; + } + } + latch.countDown(); + setLength(desc, succeeded, null); + return invocationOnMock; + }); + read(42, answer, latch, fetchTimes, succeeded); + } + + @Test + public void testReadSmallMapFromOtherThread() throws Exception { + int expectedFetchTimes = 32; + AtomicInteger fetchTimes = new AtomicInteger(0); + boolean[] succeeded = new boolean[] {true}; + Method method = IODataDesc.class.getDeclaredMethod("succeed"); + method.setAccessible(true); + CountDownLatch latch = new CountDownLatch(expectedFetchTimes); + + Answer answer = (invocationOnMock -> + { + fetchTimes.getAndIncrement(); + IODataDesc desc = invocationOnMock.getArgument(0); + desc.encode(); + method.invoke(desc); + + IODataDesc.Entry entry = desc.getEntry(0); + int offset = entry.getOffset(); + if ("0".equals(entry.getKey()) && offset == 0) { + latch.countDown(); + checkAndSetSize(desc, succeeded, 10*1024); + return invocationOnMock; + } + if ("0".equals(entry.getKey()) && offset == 10*1024) { + latch.countDown(); + checkAndSetSize(desc, succeeded, 5*1024, 15*1024); + return invocationOnMock; + } + if ("0".equals(entry.getKey()) && offset == 30*1024) { + latch.countDown(); + checkAndSetSize(desc, succeeded, 15*1024, 5*1024); + return invocationOnMock; + } + latch.countDown(); + setLength(desc, succeeded, null); + return invocationOnMock; + }); + read(42, answer, latch, fetchTimes, succeeded); + } + + private void read(int maps, Answer answer, + CountDownLatch latch, AtomicInteger fetchTimes, boolean[] succeeded) throws Exception { + UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser("test")); + SparkConf testConf = new SparkConf(false); + long minSize = 10; + testConf.set(package$.MODULE$.SHUFFLE_DAOS_READ_MINIMUM_SIZE(), minSize); + SparkContext context = new SparkContext("local", "test", testConf); + TaskContext taskContext = TaskContextObj.emptyTaskContext(); + ShuffleReadMetricsReporter metrics = taskContext.taskMetrics().createTempShuffleReadMetrics(); + // daos object mock + DaosObjectId id = Mockito.mock(DaosObjectId.class); + Mockito.when(id.isEncoded()).thenReturn(true); + DaosObjClient client = PowerMockito.mock(DaosObjClient.class); + Constructor objectConstructor = + DaosObject.class.getDeclaredConstructor(DaosObjClient.class, DaosObjectId.class); + objectConstructor.setAccessible(true); + DaosObject daosObject = Mockito.spy(objectConstructor.newInstance(client, id)); + + Mockito.doAnswer(answer).when(daosObject).fetch(any(IODataDesc.class)); + + BoundThreadExecutors executors = new BoundThreadExecutors("read_executors", 1, + new DaosReader.ReadThreadFactory()); + DaosReader daosReader = new DaosReader(daosObject, executors); + LinkedHashMap, Tuple3> partSizeMap = new LinkedHashMap<>(); + int shuffleId = 10; + int reduceId = 1; + int size = (int)(minSize + 5) * 1024; + for (int i = 0; i < maps; i++) { + partSizeMap.put(new Tuple2<>(Long.valueOf(i), 10), new Tuple3<>(Long.valueOf(size), + new ShuffleBlockId(shuffleId, i, reduceId), + BlockManagerId.apply("1", "localhost", 2, Option.empty()))); + } + + DaosShuffleInputStream is = new DaosShuffleInputStream(daosReader, partSizeMap, 2 * minSize * 1024, + 2 * 1024 * 1024, metrics); + try { + // verify cancelled task and continuing submission + for (int i = 0; i < maps; i++) { + byte[] bytes = new byte[size]; + is.read(bytes); + for (int j = 0; j < 255; j++) { + try { + Assert.assertEquals((byte) j, bytes[j]); + } catch (Throwable e) { + LOG.error("error at map " + i + ", loc: " + j); + throw e; + } + } + Assert.assertEquals(-1, is.read()); + is.nextMap(); + } + boolean alldone = latch.await(10, TimeUnit.SECONDS); + System.out.println("fetch times: " + fetchTimes.get()); + Assert.assertTrue(alldone); + Assert.assertTrue(succeeded[0]); + TaskContextObj.mergeReadMetrics(taskContext); + System.out.println("total fetch wait time: " + + taskContext.taskMetrics().shuffleReadMetrics()._fetchWaitTime().sum()); + } finally { + daosReader.close(); + is.close(true); + context.stop(); + if (executors != null) { + executors.stop(); + } + } + } + + private void checkAndSetSize(IODataDesc desc, boolean[] succeeded, int... sizes) { + if (desc.getNbrOfEntries() != sizes.length) { + succeeded[0] = false; + throw new IllegalStateException("number of entries should be " + sizes.length +", not " + desc.getNbrOfEntries()); + } + + setLength(desc, succeeded, sizes); + } + + private void setLength(IODataDesc desc, boolean[] succeeded, int[] sizes) { + for (int i = 0; i < desc.getNbrOfEntries(); i++) { + ByteBuf buf = desc.getEntry(i).getFetchedData(); + if (sizes != null) { + if (buf.capacity() != sizes[i]) { + succeeded[0] = false; + throw new IllegalStateException("buf capacity should be " + sizes[i] + ", not " + buf.capacity()); + } + } + for (int j = 0; j < buf.capacity(); j++) { + buf.writeByte((byte)j); + } + } + } + + @AfterClass + public static void afterAll() { + } +} diff --git a/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosWriterTest.java b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosWriterTest.java new file mode 100644 index 00000000..60c45269 --- /dev/null +++ b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosWriterTest.java @@ -0,0 +1,156 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos; + +import io.daos.obj.DaosObjClient; +import io.daos.obj.DaosObject; +import io.daos.obj.DaosObjectId; +import io.daos.obj.IODataDesc; +import org.apache.spark.SparkConf; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.SuppressStaticInitializationFor; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Method; +import java.util.HashMap; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.mockito.ArgumentMatchers.any; + +@RunWith(PowerMockRunner.class) +@PowerMockIgnore("javax.management.*") +@SuppressStaticInitializationFor("io.daos.obj.DaosObjClient") +public class DaosWriterTest { + + @Test + public void testGetLensWithAllEmptyPartitions() { + DaosWriter.WriteConfig writeConfig = DaosShuffleIO.loadWriteConfig(new SparkConf(false)); + DaosWriter.WriteParam param = new DaosWriter.WriteParam(); + int numPart = 10; + param.numPartitions(numPart) + .shuffleId(1) + .mapId(1) + .config(writeConfig); + DaosWriter writer = new DaosWriter(param, null, null); + long[] lens = writer.getPartitionLens(numPart); + Assert.assertEquals(numPart, lens.length); + for (int i = 0; i < numPart; i++) { + Assert.assertEquals(0, lens[i]); + } + writer.close(); + } + + @Test + public void testGetLensWithPartialEmptyPartitions() throws Exception { + DaosObjectId id = Mockito.mock(DaosObjectId.class); + Mockito.when(id.isEncoded()).thenReturn(true); + DaosObjClient client = PowerMockito.mock(DaosObjClient.class); + Constructor objectConstructor = + DaosObject.class.getDeclaredConstructor(DaosObjClient.class, DaosObjectId.class); + objectConstructor.setAccessible(true); + DaosObject daosObject = Mockito.spy(objectConstructor.newInstance(client, id)); + + Mockito.doNothing().when(daosObject).update(any(IODataDesc.class)); + + DaosWriter.WriteConfig writeConfig = DaosShuffleIO.loadWriteConfig(new SparkConf(false)); + DaosWriter.WriteParam param = new DaosWriter.WriteParam(); + int numPart = 10; + param.numPartitions(numPart) + .shuffleId(1) + .mapId(1) + .config(writeConfig); + DaosWriter writer = new DaosWriter(param, daosObject, null); + Map expectedLens = new HashMap<>(); + Random random = new Random(); + for (int i = 0; i < 5; i++) { + int idx = Math.abs(random.nextInt(numPart)); + while (expectedLens.containsKey(idx)) { + idx = Math.abs(random.nextInt(numPart)); + } + int size = Math.abs(random.nextInt(8000)); + writer.write(idx, new byte[size]); + writer.flush(idx); + expectedLens.put(idx, size); + } + long[] lens = writer.getPartitionLens(numPart); + Assert.assertEquals(numPart, lens.length); + for (int i = 0; i < numPart; i++) { + int expected = expectedLens.get(i) == null ? 0 : expectedLens.get(i); + Assert.assertEquals(expected, (int)(lens[i])); + } + writer.close(); + } + + @Test(expected = IllegalStateException.class) + public void testWriteTaskFailed() throws Exception { + DaosObjectId id = Mockito.mock(DaosObjectId.class); + Mockito.when(id.isEncoded()).thenReturn(true); + DaosObjClient client = PowerMockito.mock(DaosObjClient.class); + Constructor objectConstructor = + DaosObject.class.getDeclaredConstructor(DaosObjClient.class, DaosObjectId.class); + objectConstructor.setAccessible(true); + DaosObject daosObject = Mockito.spy(objectConstructor.newInstance(client, id)); + + AtomicInteger counter = new AtomicInteger(0); + Method method = IODataDesc.class.getDeclaredMethod("succeed"); + method.setAccessible(true); + Mockito.doAnswer(invoc -> { + IODataDesc desc = invoc.getArgument(0); + desc.encode(); + counter.incrementAndGet(); + if (counter.get() == 5) { + method.invoke(desc); + } + return invoc; + }).when(daosObject).update(any(IODataDesc.class)); + + DaosWriter.WriteConfig writeConfig = DaosShuffleIO.loadWriteConfig(new SparkConf(false)); + DaosWriter.WriteParam param = new DaosWriter.WriteParam(); + int numPart = 10; + param.numPartitions(numPart) + .shuffleId(1) + .mapId(1) + .config(writeConfig); + + BoundThreadExecutors executors = new BoundThreadExecutors("read_executors", 1, + new DaosReader.ReadThreadFactory()); + DaosWriter writer = new DaosWriter(param, daosObject, executors.nextExecutor()); + for (int i = 0; i < numPart; i++) { + writer.write(i, new byte[100]); + writer.flush(i); + } + + writer.close(); + + executors.stop(); + } +} diff --git a/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleReaderSuite.scala b/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleReaderSuite.scala new file mode 100644 index 00000000..6a9591ad --- /dev/null +++ b/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleReaderSuite.scala @@ -0,0 +1,212 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos + +import java.io.ByteArrayOutputStream +import java.util + +import io.daos.BufferAllocator +import io.daos.obj.{DaosObject, IODataDesc} +import org.mockito._ +import org.mockito.Mockito.{doNothing, mock, when} + +import org.apache.spark._ +import org.apache.spark.internal.config +import org.apache.spark.serializer.{JavaSerializer, SerializerManager} +import org.apache.spark.shuffle.BaseShuffleHandle +import org.apache.spark.storage.{BlockManagerId, ShuffleBlockId} + +class DaosShuffleReaderSuite extends SparkFunSuite with LocalSparkContext { + + override def beforeAll(): Unit = { + super.beforeAll() + logInfo("start executors in DaosReader " + classOf[DaosReader]) + MockitoAnnotations.initMocks(this) + } + + private def mockObjectsForSingleDaosCall(reduceId: Int, numMaps: Int, byteOutputStream: ByteArrayOutputStream): + (DaosReader, DaosShuffleIO, DaosObject) = { + // mock + val daosReader: DaosReader = Mockito.mock(classOf[DaosReader]) + val daosObject = Mockito.mock(classOf[DaosObject]) + val shuffleIO = Mockito.mock(classOf[DaosShuffleIO]) + + val desc = Mockito.mock(classOf[IODataDesc]) + when(daosObject.createDataDescForFetch(String.valueOf(reduceId), IODataDesc.IodType.ARRAY, 1)) + .thenReturn(desc) + doNothing().when(daosObject).fetch(desc) + when(desc.getNbrOfEntries()).thenReturn(numMaps) + + (0 until numMaps).foreach(i => { + val entry = Mockito.mock(classOf[desc.Entry]) + when(desc.getEntry(i)).thenReturn(entry) + val buf = BufferAllocator.objBufWithNativeOrder(byteOutputStream.size()) + buf.writeBytes(byteOutputStream.toByteArray) + when(entry.getFetchedData).thenReturn(buf) + when(entry.getKey).thenReturn(String.valueOf(i)) + }) + (daosReader, shuffleIO, daosObject) + } + + private def mockObjectsForMultipleDaosCall(reduceId: Int, numMaps: Int, byteOutputStream: ByteArrayOutputStream, + executors: BoundThreadExecutors): + (DaosReader, DaosShuffleIO, DaosObject) = { + // mock + val daosObject = Mockito.mock(classOf[DaosObject]) + val daosReader: DaosReader = + if (executors != null) Mockito.spy(new DaosReader(daosObject, executors)) else Mockito.mock(classOf[DaosReader]) + val shuffleIO = Mockito.mock(classOf[DaosShuffleIO]) + val descList = new util.ArrayList[IODataDesc] + + (0 until numMaps).foreach(_ => { + descList.add(Mockito.mock(classOf[IODataDesc])) + }) + val times = Array[Int](1) + times(0) = 0 + when(daosObject.createDataDescForFetch(String.valueOf(reduceId), IODataDesc.IodType.ARRAY, 1)) + .thenAnswer(i => { + val desc = descList.get(times(0)) + times(0) += 1 + desc + }) + + (0 until numMaps).foreach(i => { + val desc = descList.get(i) + doNothing().when(daosObject).fetch(desc) + when(desc.getNbrOfEntries()).thenReturn(1) + when(desc.isSucceeded()).thenReturn(true) + when(desc.getTotalRequestSize).thenReturn(byteOutputStream.toByteArray.length) + val entry = Mockito.mock(classOf[desc.Entry]) + when(desc.getEntry(0)).thenReturn(entry) + val buf = BufferAllocator.objBufWithNativeOrder(byteOutputStream.size()) + buf.writeBytes(byteOutputStream.toByteArray) + when(entry.getFetchedData).thenReturn(buf) + when(entry.getKey).thenReturn(String.valueOf(i)) + }) + (daosReader, shuffleIO, daosObject) + } + + private def testRead(keyValuePairsPerMap: Int, numMaps: Int, singleCall: Boolean = true, + executors: BoundThreadExecutors = null): Unit = { + val testConf = new SparkConf(false) + testConf.set(SHUFFLE_DAOS_READ_FROM_OTHER_THREAD, executors != null) + + // Create a buffer with some randomly generated key-value pairs to use as the shuffle data + // from each mappers (all mappers return the same shuffle data). + val serializer = new JavaSerializer(testConf) + val byteOutputStream = new ByteArrayOutputStream() + val serializationStream = serializer.newInstance().serializeStream(byteOutputStream) + (0 until keyValuePairsPerMap).foreach { i => + serializationStream.writeKey(i) + serializationStream.writeValue(2*i) + } + + if (!singleCall) { + val value = math.ceil(byteOutputStream.toByteArray.length.toDouble / 1024).toInt + testConf.set(SHUFFLE_DAOS_READ_MAX_BYTES_IN_FLIGHT, value.toLong) + testConf.set(SHUFFLE_DAOS_READ_MINIMUM_SIZE, value.toLong) + } + + val reduceId = 15 + val shuffleId = 22 + // Create a SparkContext as a convenient way of setting SparkEnv (needed because some of the + // shuffle code calls SparkEnv.get()). + sc = new SparkContext("local", "test", testConf) + + // Make a mocked MapOutputTracker for the shuffle reader to use to determine what + // shuffle data to read. + val mapOutputTracker = mock(classOf[MapOutputTracker]) + val localBlockManagerId = BlockManagerId("test-client", "test-client", 1) + when(mapOutputTracker.getMapSizesByExecutorId( + shuffleId, reduceId, reduceId + 1)).thenReturn { + // Test a scenario where all data is local, to avoid creating a bunch of additional mocks + // for the code to read data over the network. + val shuffleBlockIdsAndSizes = (0 until numMaps).map { mapId => + val shuffleBlockId = ShuffleBlockId(shuffleId, mapId, reduceId) + (shuffleBlockId, byteOutputStream.size().toLong, mapId) + } + Seq((localBlockManagerId, shuffleBlockIdsAndSizes)).toIterator + } + + // Create a mocked shuffle handle to pass into HashShuffleReader. + val shuffleHandle = { + val dependency = mock(classOf[ShuffleDependency[Int, Int, Int]]) + when(dependency.serializer).thenReturn(serializer) + when(dependency.aggregator).thenReturn(None) + when(dependency.keyOrdering).thenReturn(None) + new BaseShuffleHandle(shuffleId, dependency) + } + + val serializerManager = new SerializerManager( + serializer, + new SparkConf() + .set(config.SHUFFLE_COMPRESS, false) + .set(config.SHUFFLE_SPILL_COMPRESS, false)) + + val taskContext = TaskContext.empty() + val metrics = taskContext.taskMetrics.createTempShuffleReadMetrics() + val blocksByAddress = mapOutputTracker.getMapSizesByExecutorId( + shuffleId, reduceId, reduceId + 1) + + val (daosReader, shuffleIO, daosObject) = + if (singleCall) { + mockObjectsForSingleDaosCall(reduceId, numMaps, byteOutputStream) + } else { + mockObjectsForMultipleDaosCall(reduceId, numMaps, byteOutputStream, executors) + } + + when(shuffleIO.getDaosReader(shuffleId)).thenReturn(daosReader) + when(daosReader.getObject).thenReturn(daosObject) + + val shuffleReader = new DaosShuffleReader[Int, Int]( + shuffleHandle, + blocksByAddress, + taskContext, + metrics, + shuffleIO, + serializerManager, + false) + + assert(shuffleReader.read().length === keyValuePairsPerMap * numMaps) + + // verify metrics + assert(metrics.remoteBytesRead === numMaps * byteOutputStream.toByteArray.length) + logInfo("remotes bytes: " + metrics.remoteBytesRead) + assert(metrics.remoteBlocksFetched === numMaps) + } + + test("test reader daos once") { + testRead(10, 6) + } + + test("test reader daos multiple times") { + testRead(7168, 4, false) + } + + test("test reader daos multiple times from other thread") { + val executors = new BoundThreadExecutors("read_executors", 1, new DaosReader.ReadThreadFactory) + testRead(7168, 6, false, executors) + executors.stop() + } +} diff --git a/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleWriterPerf.scala b/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleWriterPerf.scala new file mode 100644 index 00000000..f389ee36 --- /dev/null +++ b/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleWriterPerf.scala @@ -0,0 +1,106 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos + +import org.mockito.{Mock, Mockito, MockitoAnnotations} +import org.mockito.Answers._ +import org.mockito.Mockito.{mock, when} +import org.scalatest.Matchers +import scala.collection.mutable +import scala.util.Random + +import org.apache.spark.{Partitioner, SharedSparkContext, ShuffleDependency, SparkFunSuite} +import org.apache.spark.memory.MemoryTestingUtils +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.shuffle.BaseShuffleHandle +import org.apache.spark.util.Utils + +class DaosShuffleWriterPerf extends SparkFunSuite with SharedSparkContext with Matchers { + + @Mock(answer = RETURNS_SMART_NULLS) + private var shuffleIO: DaosShuffleIO = _ + + private val shuffleId = 0 + private val numMaps = 1000 + private var shuffleHandle: BaseShuffleHandle[Int, Array[Byte], Array[Byte]] = _ + private val serializer = new JavaSerializer(conf) + + private val singleBufSize = conf.get(SHUFFLE_DAOS_WRITE_SINGLE_BUFFER_SIZE) * 1024 * 1024 + private val minSize = conf.get(SHUFFLE_DAOS_WRITE_MINIMUM_SIZE) * 1024 + + conf.set(SHUFFLE_DAOS_WRITE_PARTITION_BUFFER_SIZE, 100L) + conf.set(SHUFFLE_DAOS_WRITE_BUFFER_SIZE, 80L) + + override def beforeEach(): Unit = { + super.beforeEach() + MockitoAnnotations.initMocks(this) + val partitioner = new Partitioner() { + def numPartitions = numMaps + + def getPartition(key: Any) = Utils.nonNegativeMod(key.hashCode, numPartitions) + } + shuffleHandle = { + val dependency = mock(classOf[ShuffleDependency[Int, Array[Byte], Array[Byte]]]) + when(dependency.partitioner).thenReturn(partitioner) + when(dependency.serializer).thenReturn(serializer) + when(dependency.aggregator).thenReturn(None) + when(dependency.keyOrdering).thenReturn(None) + new BaseShuffleHandle(shuffleId, dependency) + } + } + + test("write with some records") { + val context = MemoryTestingUtils.fakeTaskContext(sc.env) + val bytes = new Array[Byte](100) + (0 until 100).foreach(i => bytes(i) = (i + 1).toByte) + val records = new mutable.MutableList[(Int, Array[Byte])]() + val size = 128 * 1024 * 1024 + var count = 0 + var index = 0 + while (count < size) { + records += ((index, bytes)) + index += 1 + count += 100 + } + +// val records = List[(Int, Int)]((1, 2), (2, 3), (4, 4), (6, 5)) + + val daosWriter: DaosWriter = Mockito.mock(classOf[DaosWriter]) + when(shuffleIO.getDaosWriter(numMaps, shuffleId, context.taskAttemptId())) + .thenReturn(daosWriter) + val partitionLengths = Array[Long](numMaps) + when(daosWriter.getPartitionLens(numMaps)).thenReturn(partitionLengths) + + val writer = new DaosShuffleWriter[Int, Array[Byte], Array[Byte]](shuffleHandle, shuffleId, context, shuffleIO) + val start = System.currentTimeMillis() + writer.write(records.map(k => { + val p = new Random(util.hashing.byteswap32(k._1)).nextInt(numMaps) + (p, k._2) + }).iterator) + logInfo(s"time: ${System.currentTimeMillis() - start}") + writer.stop(success = true) + val writeMetrics = context.taskMetrics().shuffleWriteMetrics + assert(records.size === writeMetrics.recordsWritten) + } +} diff --git a/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleWriterSuite.scala b/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleWriterSuite.scala new file mode 100644 index 00000000..d669aee7 --- /dev/null +++ b/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleWriterSuite.scala @@ -0,0 +1,108 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos + +import org.mockito.{Mock, Mockito, MockitoAnnotations} +import org.mockito.Answers._ +import org.mockito.ArgumentMatchers.{any, anyInt} +import org.mockito.Mockito.{mock, never, when} +import org.scalatest.Matchers +import scala.collection.mutable + +import org.apache.spark.{Partitioner, SharedSparkContext, ShuffleDependency, SparkFunSuite} +import org.apache.spark.memory.MemoryTestingUtils +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.shuffle.BaseShuffleHandle +import org.apache.spark.util.Utils + +class DaosShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with Matchers { + + @Mock(answer = RETURNS_SMART_NULLS) + private var shuffleIO: DaosShuffleIO = _ + + private val shuffleId = 0 + private val numMaps = 5 + private var shuffleHandle: BaseShuffleHandle[Int, Int, Int] = _ + private val serializer = new JavaSerializer(conf) + + private val singleBufSize = conf.get(SHUFFLE_DAOS_WRITE_SINGLE_BUFFER_SIZE) * 1024 * 1024 + private val minSize = conf.get(SHUFFLE_DAOS_WRITE_MINIMUM_SIZE) * 1024 + + override def beforeEach(): Unit = { + super.beforeEach() + MockitoAnnotations.initMocks(this) + val partitioner = new Partitioner() { + def numPartitions = numMaps + + def getPartition(key: Any) = Utils.nonNegativeMod(key.hashCode, numPartitions) + } + shuffleHandle = { + val dependency = mock(classOf[ShuffleDependency[Int, Int, Int]]) + when(dependency.partitioner).thenReturn(partitioner) + when(dependency.serializer).thenReturn(serializer) + when(dependency.aggregator).thenReturn(None) + when(dependency.keyOrdering).thenReturn(None) + new BaseShuffleHandle(shuffleId, dependency) + } + } + + test("write empty data") { + val context = MemoryTestingUtils.fakeTaskContext(sc.env) + val daosWriter: DaosWriter = Mockito.mock(classOf[DaosWriter]) + when(shuffleIO.getDaosWriter(5, shuffleId, context.taskAttemptId())) + .thenReturn(daosWriter) + val partitionLengths = Array[Long](5) + when(daosWriter.getPartitionLens(numMaps)).thenReturn(partitionLengths) + + val writer = new DaosShuffleWriter[Int, Int, Int](shuffleHandle, shuffleId, context, shuffleIO) + writer.write(Iterator.empty) + writer.stop(success = true) + val writeMetrics = context.taskMetrics().shuffleWriteMetrics + assert(writeMetrics.bytesWritten === 0) + assert(writeMetrics.recordsWritten === 0) + + Mockito.verify(daosWriter, never()).write(any, anyInt()) + Mockito.verify(daosWriter, never()).write(any, any(classOf[Array[Byte]])) + } + + test("write with some records") { + val context = MemoryTestingUtils.fakeTaskContext(sc.env) + val dataList = new mutable.MutableList[(Int, Array[Byte])]() + + val records = List[(Int, Int)]((1, 2), (2, 3), (4, 4), (6, 5)) + + val daosWriter: DaosWriter = Mockito.mock(classOf[DaosWriter]) + when(shuffleIO.getDaosWriter(5, shuffleId, context.taskAttemptId())) + .thenReturn(daosWriter) + val partitionLengths = Array[Long](5) + when(daosWriter.getPartitionLens(numMaps)).thenReturn(partitionLengths) + + val writer = new DaosShuffleWriter[Int, Int, Int](shuffleHandle, shuffleId, context, shuffleIO) + writer.write(records.toIterator) + writer.stop(success = true) + val writeMetrics = context.taskMetrics().shuffleWriteMetrics + assert(395 === writeMetrics.bytesWritten) + assert(records.size === writeMetrics.recordsWritten) + } +} diff --git a/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/SizeSamplerSuite.scala b/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/SizeSamplerSuite.scala new file mode 100644 index 00000000..c7ded396 --- /dev/null +++ b/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/SizeSamplerSuite.scala @@ -0,0 +1,103 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos + +import scala.util.Random + +import org.apache.spark.SparkFunSuite + +class SizeSamplerSuite extends SparkFunSuite { + + test("test sample AppendOnlyMap by update") { + val stat = new SampleStat + var grew = false; + val map = new SizeSamplerAppendOnlyMap[Int, Int](stat) { + override def growTable(): Unit = { + super.growTable() + grew = true + } + } + (1 to 65).foreach { + i => map.update(i, i) + assert(i === stat.numUpdates) + assert(stat.lastNumUpdates <= stat.numUpdates) + assert(stat.nextSampleNum >= stat.numUpdates) + if (i == 15) { + assert(stat.nextSampleNum === 17) + } + if (i == 45) { + assert(grew === true) + } + } + } + + test("test sample AppendOnlyMap by changeValue") { + val stat = new SampleStat + var grew = false; + val map = new SizeSamplerAppendOnlyMap[Int, Int](stat) { + override def growTable(): Unit = { + super.growTable() + grew = true + } + } + val updateFun = (exist: Boolean, v: Int) => { + new Random().nextInt(100) + v + } + (1 to 65).foreach { + i => map.changeValue(i, updateFun) + assert(i === stat.numUpdates) + assert(stat.lastNumUpdates <= stat.numUpdates) + assert(stat.nextSampleNum >= stat.numUpdates) + if (i == 15) { + assert(stat.nextSampleNum === 17) + } + if (i == 45) { + assert(grew === true) + } + } + } + + test("test sample PairBuffer by insert") { + val stat = new SampleStat + var grew = false; + val buffer = new SizeSamplerPairBuffer[Int, Int](stat) { + override def growArray(): Unit = { + super.growArray() + grew = true + } + } + (1 to 73).foreach { + i => buffer.insert(i, i) + assert(i === stat.numUpdates) + assert(stat.lastNumUpdates <= stat.numUpdates) + assert(stat.nextSampleNum >= stat.numUpdates) + if (i == 15) { + assert(stat.nextSampleNum === 17) + } + if (i == 65) { + assert(grew === true) + } + } + } +} diff --git a/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/SizeTest.scala b/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/SizeTest.scala new file mode 100644 index 00000000..7f231e1e --- /dev/null +++ b/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/SizeTest.scala @@ -0,0 +1,45 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos + +import org.junit.Test + +class SizeTest { + + def deCompressSize(size: Int): Long = { + val LOG_BASE = 1.1 + val bytes = math.min(255, math.ceil(math.log(size) / math.log(LOG_BASE)).toInt).toByte + val deSize = math.pow(LOG_BASE, bytes & 0xFF).toLong + deSize + } + + @Test + def testCompressSize(): Unit = { + (2 to 100).foreach(size => { + val deSize = deCompressSize(size) + assert(size <= deSize && deSize <= size*1.1) + }) + } + +} diff --git a/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/TaskContextObj.scala b/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/TaskContextObj.scala new file mode 100644 index 00000000..e2facc47 --- /dev/null +++ b/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/TaskContextObj.scala @@ -0,0 +1,37 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos + +import org.apache.spark.TaskContext + +object TaskContextObj { + + def emptyTaskContext: TaskContext = { + TaskContext.empty() + } + + def mergeReadMetrics(taskContext: TaskContext): Unit = { + taskContext.taskMetrics().mergeShuffleReadMetrics() + } +} diff --git a/CHANGELOG.md b/shuffle-hadoop/CHANGELOG.md similarity index 100% rename from CHANGELOG.md rename to shuffle-hadoop/CHANGELOG.md diff --git a/shuffle-hadoop/README.md b/shuffle-hadoop/README.md new file mode 100644 index 00000000..2f49f242 --- /dev/null +++ b/shuffle-hadoop/README.md @@ -0,0 +1,214 @@ +# Remote Shuffle Based on Hadoop Filesystem + +## Online Documentation + +You can find the all the PMem Spill documents on the [project web page](https://oap-project.github.io/remote-shuffle/). + +## Contents +- [Introduction](#introduction) +- [User Guide](#userguide) + +## Introduction +Remote Shuffle is a Spark* ShuffleManager plugin, shuffling data through a remote Hadoop-compatible file system, as opposed to vanilla Spark's local-disks. + +This is an essential part of enabling Spark on disaggregated compute and storage architecture. + + +### Installation +We have provided a Conda package which will automatically install dependencies needed by OAP, you can refer to [OAP-Installation-Guide](./docs/OAP-Installation-Guide.md) for more information. If you have finished [OAP-Installation-Guide](./docs/OAP-Installation-Guide.md), you can find compiled OAP jars in `$HOME/miniconda2/envs/oapenv/oap_jars/`. + +## Developer Guide +### Build and Deploy + +We have provided a Conda package which will automatically install dependencies needed by OAP, you can refer to [OAP-Installation-Guide](./docs/OAP-Installation-Guide.md) for more information. If you have finished [OAP-Installation-Guide](./docs/OAP-Installation-Guide.md), you can find compiled remote shuffle jars under `$HOME/miniconda2/envs/oapenv/oap_jars`. +Then just skip this section and jump to [User Guide](#user-guide). + +Build this module using the following command in `OAP/oap-shuffle/remote-shuffle` folder. This file needs to be deployed on every compute node that runs Spark. Manually place it on all nodes or let resource manager do the work. + +``` + mvn -DskipTests clean package +``` +## User Guide +### Enable Remote Shuffle + +Add the `.jar` files to the classpath of Spark driver and executors: Put the +following configurations in spark-defaults.conf or Spark submit command line arguments. + +Note: For DAOS users, DAOS Hadoop/Java API jars should also be included in the classpath as we leverage DAOS Hadoop filesystem. + +``` + spark.executor.extraClassPath $HOME/miniconda2/envs/oapenv/oap_jars/remote-shuffle-.jar + spark.driver.extraClassPath $HOME/miniconda2/envs/oapenv/oap_jars/remote-shuffle-.jar +``` + +Enable the remote shuffle manager and specify the Hadoop storage system URI holding shuffle data. + +``` + spark.shuffle.manager org.apache.spark.shuffle.remote.RemoteShuffleManager + spark.shuffle.remote.storageMasterUri daos://default:1 # Or hdfs://namenode:port, file:///my/shuffle/dir +``` + +### Configurations + +Configurations and tuning parameters that change the behavior of remote shuffle. Most of them should work well under default values. + +#### Shuffle Root Directory + +This is to configure the root directory holding remote shuffle files. For each Spark application, a +directory named after application ID is created under this root directory. + +``` + spark.shuffle.remote.filesRootDirectory /shuffle +``` + +#### Index Cache Size + +This is to configure the cache size for shuffle index files per executor. Shuffle data includes data files and +index files. An index file is small but will be read many (the number of reducers) times. On a large scale, constantly +reading these small index files from Hadoop Filesystem implementation(i.e. HDFS) is going to cause much overhead and latency. In addition, the shuffle files’ +transfer completely relies on the network between compute nodes and storage nodes. But the network inside compute nodes are +not fully utilized. The index cache can eliminate the overhead of reading index files from storage cluster multiple times. By +enabling index file cache, a reduce task fetches them from the remote executors who write them instead of reading from +storage. If the remote executor doesn’t have a desired index file in its cache, it will read the file from storage and cache +it locally. The feature can also be disabled by setting the value to zero. + +``` + spark.shuffle.remote.index.cache.size 30m +``` + +#### Number of Threads Reading Data Files + +This is one of the parameters influencing shuffle read performance. It is to determine number of threads per executor reading shuffle data files from storage. + +``` + spark.shuffle.remote.numReadThreads 5 +``` + +#### Number of Threads Transitioning Index Files (when index cache is enabled) + +This is one of the parameters influencing shuffle read performance. It is to determine the number of client and server threads that transmit index information from another executor’s cache. It is only valid when the index cache feature is enabled. + +``` + spark.shuffle.remote.numIndexReadThreads 3 +``` + +#### Bypass-merge-sort Threshold + +This threshold is used to decide using bypass-merge(hash-based) shuffle or not. By default we disable(by setting it to -1) +hash-based shuffle writer in remote shuffle, because when memory is relatively sufficient, sort-based shuffle writer is often more efficient than the hash-based one. +Hash-based shuffle writer entails a merging process, performing 3x I/Os than total shuffle size: 1 time for read I/Os and 2 times for write I/Os, this can be an even larger overhead under remote shuffle: +the 3x shuffle size is gone through network, arriving at a remote storage system. + +``` + spark.shuffle.remote.bypassMergeThreshold -1 +``` + +#### Configurations fetching port for HDFS + +When the backend storage is HDFS, we contact http://$host:$port/conf to fetch configurations. They were not locally loaded because we assume absence of local storage. + +``` + spark.shuffle.remote.hdfs.storageMasterUIPort 50070 +``` + +#### Inherited Spark Shuffle Configurations + +These configurations are inherited from upstream Spark, they are still supported in remote shuffle. More explanations can be found in [Spark core docs](https://spark.apache.org/docs/3.0.0/configuration.html#shuffle-behavior) and [Spark SQL docs](https://spark.apache.org/docs/2.4.4/sql-performance-tuning.html). +``` + spark.reducer.maxSizeInFlight + spark.reducer.maxReqsInFlight + spark.reducer.maxBlocksInFlightPerAddress + spark.shuffle.compress + spark.shuffle.file.buffer + spark.shuffle.io.maxRetries + spark.shuffle.io.numConnectionsPerPeer + spark.shuffle.io.preferDirectBufs + spark.shuffle.io.retryWait + spark.shuffle.io.backLog + spark.shuffle.spill.compress + spark.shuffle.accurateBlockThreshold + spark.sql.shuffle.partitions +``` + +#### Deprecated Spark Shuffle Configurations + +These configurations are deprecated and will not take effect. +``` + spark.shuffle.sort.bypassMergeThreshold # Replaced by spark.shuffle.remote.bypassMergeThreshold + spark.maxRemoteBlockSizeFetchToMem # As we assume no local disks on compute nodes, shuffle blocks are all fetched to memory + + spark.shuffle.service.enabled # All following configurations are related to External Shuffle Service. ESS & remote shuffle cannot be enabled at the same time, as this remote shuffle facility takes over almost all functionalities of ESS. + spark.shuffle.service.port + spark.shuffle.service.index.cache.size + spark.shuffle.maxChunksBeingTransferred + spark.shuffle.registration.timeout + spark.shuffle.registration.maxAttempts +``` + +### Performance Evaluation Tool + +Leverage this tool to evaluate shuffle write/read performance separately under your specific storage system. This tool starts one Java process with #poolSize number of threads, running the specified remote-shuffle writers/readers in this module. Additional Spark configurations can be put in "./spark-defaults.conf" and will be loaded.(and printed as part of the summary for recording) + +Configuration details: +* `-h` or `--help`: display help messages +* `-m` or `--mappers`: the number of mappers, default to 5 +* `-r` or `--reducers`: the number of reducers, default to 5 +* `-p` or `--poolSize`: the number task threads in write/read thread pool, similar to spark.executor.cores. e.g. if mappers=15, poolSize=5, it takes 3 rounds to finish this job +* `-n` or `--rows`: the number of rows per mapper, default to 1000 +* `-b` or `--shuffleBlockRawSize`: the size of each shuffle block, default to 20000 Bytes +* `-w` or `--writer`: the type of shuffle writers for benchmark, can be one of general, unsafe and bypassmergesort, default to unsafe +* `-onlyWrite` or `--onlyWrite`: containing this flag then the benchmark only includes shuffle write stage, default behavior is perform both write & read +* `-uri` or `--storageMasterUri`: Hadoop-compatible storage Master URI, default to file:// +* `-d` or `--dir`: Shuffle directory, default /tmp +* `-l` or `--log`: Log level, default to WARN + + +Sample command: +``` +java -cp target/remote-shuffle-0.1-SNAPSHOT-test-jar-with-dependencies.jar org.apache.spark.shuffle.remote.PerformanceEvaluationTool -h +``` + +Sample output +``` +unsafe shuffle writer: + raw total size: 123 GB + compressed size: 135 GB + duration: 88.3 seconds + + throughput(raw): 1429.06843144412 MB/s + throughput(storage): 1570.9931870053674 MB/s + + number of mappers: 210 + number of reducers: 70 + block size(raw): 8 MB + block size(storage): 9 MB + + properties: spark.reducer.maxSizeInFlight -> 100m, spark.shuffle.remote.numReadThreads -> 8, spark.shuffle.remote.reducer.maxBlocksInFlightPerAddress -> 3 + + records per mapper: 70 + load size per record:9000000 + + shuffle storage daos://default:1 + shuffle folder: /tmp/shuffle +------------------------------------------------------------------------------------------------------------------------- +shuffle reader: + raw total size: 123 GB + compressed size: 135 GB + duration: 49.8 seconds + + throughput(raw): 2533.665772753123 MB/s + throughput(storage): 2785.2911586057153 MB/s + + number of mappers: 210 + number of reducers: 70 + block size(raw): 8 MB + block size(storage): 9 MB + + properties: spark.reducer.maxSizeInFlight -> 100m, spark.shuffle.remote.numReadThreads -> 8, spark.shuffle.remote.reducer.maxBlocksInFlightPerAddress -> 3 + + records per mapper: 70 + load size per record:9000000 + + shuffle storage daos://default:1 + shuffle folder: /tmp/shuffle +``` diff --git a/dev/checkstyle.xml b/shuffle-hadoop/dev/checkstyle.xml similarity index 100% rename from dev/checkstyle.xml rename to shuffle-hadoop/dev/checkstyle.xml diff --git a/dev/post_results_to_PR.sh b/shuffle-hadoop/dev/post_results_to_PR.sh similarity index 100% rename from dev/post_results_to_PR.sh rename to shuffle-hadoop/dev/post_results_to_PR.sh diff --git a/docs/OAP-Developer-Guide.md b/shuffle-hadoop/docs/OAP-Developer-Guide.md similarity index 100% rename from docs/OAP-Developer-Guide.md rename to shuffle-hadoop/docs/OAP-Developer-Guide.md diff --git a/docs/OAP-Installation-Guide.md b/shuffle-hadoop/docs/OAP-Installation-Guide.md similarity index 100% rename from docs/OAP-Installation-Guide.md rename to shuffle-hadoop/docs/OAP-Installation-Guide.md diff --git a/docs/User-Guide.md b/shuffle-hadoop/docs/User-Guide.md similarity index 100% rename from docs/User-Guide.md rename to shuffle-hadoop/docs/User-Guide.md diff --git a/docs/index.md b/shuffle-hadoop/docs/index.md similarity index 100% rename from docs/index.md rename to shuffle-hadoop/docs/index.md diff --git a/mkdocs.yml b/shuffle-hadoop/mkdocs.yml similarity index 100% rename from mkdocs.yml rename to shuffle-hadoop/mkdocs.yml diff --git a/shuffle-hadoop/pom.xml b/shuffle-hadoop/pom.xml new file mode 100644 index 00000000..21835f40 --- /dev/null +++ b/shuffle-hadoop/pom.xml @@ -0,0 +1,262 @@ + + + 4.0.0 + + + com.intel.oap + remote-shuffle-parent + 1.1.0 + + + shuffle-hadoop + OAP Remote Shuffle Based on Hadoop Filesystem + jar + + + + Chenzhao Guo + chenzhao.guo@intel.com + + + + + UTF-8 + UTF-8 + + + + + org.scala-lang + scala-library + + + org.apache.spark + spark-core_2.12 + + + org.apache.spark + spark-core_2.12 + tests + test-jar + test + + + org.apache.spark + spark-sql_2.12 + ${spark.version} + test + + + org.apache.hadoop + hadoop-client + 2.7.4 + + + org.scalatest + scalatest_${scala.binary.version} + test + + + junit + junit + test + + + org.hamcrest + hamcrest-core + 1.3 + test + + + org.hamcrest + hamcrest-library + 1.3 + test + + + org.mockito + mockito-core + test + + + org.mock-server + mockserver-netty + 5.11.0 + + + org.mock-server + mockserver-client-java + 5.11.0 + + + org.eclipse.jetty + jetty-servlet + 9.4.12.v20180830 + test + + + commons-cli + commons-cli + 1.4 + test + + + + + + + net.alchim31.maven + scala-maven-plugin + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + ${java.version} + ${java.version} + + + + compile + + compile + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + **/Test*.java + **/*Test.java + **/*TestCase.java + **/*Suite.java + + + 1 + ${project.basedir} + + + 1 + ${scala.binary.version} + + + + + org.scalatest + scalatest-maven-plugin + + . + WDF TestSuite.txt + + 1 + ${project.basedir} + + + 1 + ${scala.binary.version} + + + + + test + + test + + + + + + org.apache.maven.plugins + maven-jar-plugin + 3.0.2 + + + + test-jar + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + false + true + + src/main/java + src/main/scala + + + src/test/java + src/test/scala + + dev/checkstyle.xml + ${basedir}/target/checkstyle-output.xml + ${project.build.sourceEncoding} + ${project.reporting.outputEncoding} + + + + compile + + check + + + + + + + + + diff --git a/scalastyle-config.xml b/shuffle-hadoop/scalastyle-config.xml similarity index 100% rename from scalastyle-config.xml rename to shuffle-hadoop/scalastyle-config.xml diff --git a/src/main/java/org/apache/spark/network/shuffle/MyOneForOneBlockFetcher.java b/shuffle-hadoop/src/main/java/org/apache/spark/network/shuffle/MyOneForOneBlockFetcher.java similarity index 100% rename from src/main/java/org/apache/spark/network/shuffle/MyOneForOneBlockFetcher.java rename to shuffle-hadoop/src/main/java/org/apache/spark/network/shuffle/MyOneForOneBlockFetcher.java diff --git a/src/main/java/org/apache/spark/shuffle/sort/RemoteBypassMergeSortShuffleWriter.java b/shuffle-hadoop/src/main/java/org/apache/spark/shuffle/sort/RemoteBypassMergeSortShuffleWriter.java similarity index 100% rename from src/main/java/org/apache/spark/shuffle/sort/RemoteBypassMergeSortShuffleWriter.java rename to shuffle-hadoop/src/main/java/org/apache/spark/shuffle/sort/RemoteBypassMergeSortShuffleWriter.java diff --git a/src/main/java/org/apache/spark/shuffle/sort/RemoteSpillInfo.java b/shuffle-hadoop/src/main/java/org/apache/spark/shuffle/sort/RemoteSpillInfo.java similarity index 100% rename from src/main/java/org/apache/spark/shuffle/sort/RemoteSpillInfo.java rename to shuffle-hadoop/src/main/java/org/apache/spark/shuffle/sort/RemoteSpillInfo.java diff --git a/src/main/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleSorter.java b/shuffle-hadoop/src/main/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleSorter.java similarity index 100% rename from src/main/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleSorter.java rename to shuffle-hadoop/src/main/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleSorter.java diff --git a/src/main/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleWriter.java b/shuffle-hadoop/src/main/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleWriter.java similarity index 100% rename from src/main/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleWriter.java rename to shuffle-hadoop/src/main/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleWriter.java diff --git a/src/main/scala/org/apache/spark/network/netty/MyNettyBlockRpcServer.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/network/netty/MyNettyBlockRpcServer.scala similarity index 100% rename from src/main/scala/org/apache/spark/network/netty/MyNettyBlockRpcServer.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/network/netty/MyNettyBlockRpcServer.scala diff --git a/src/main/scala/org/apache/spark/network/netty/RemoteShuffleTransferService.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/network/netty/RemoteShuffleTransferService.scala similarity index 100% rename from src/main/scala/org/apache/spark/network/netty/RemoteShuffleTransferService.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/network/netty/RemoteShuffleTransferService.scala diff --git a/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala similarity index 100% rename from src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala diff --git a/src/main/scala/org/apache/spark/shuffle/remote/HadoopFileSegmentManagedBuffer.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/HadoopFileSegmentManagedBuffer.scala similarity index 100% rename from src/main/scala/org/apache/spark/shuffle/remote/HadoopFileSegmentManagedBuffer.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/HadoopFileSegmentManagedBuffer.scala diff --git a/src/main/scala/org/apache/spark/shuffle/remote/RemoteAggregator.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteAggregator.scala similarity index 100% rename from src/main/scala/org/apache/spark/shuffle/remote/RemoteAggregator.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteAggregator.scala diff --git a/src/main/scala/org/apache/spark/shuffle/remote/RemoteBlockObjectWriter.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteBlockObjectWriter.scala similarity index 100% rename from src/main/scala/org/apache/spark/shuffle/remote/RemoteBlockObjectWriter.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteBlockObjectWriter.scala diff --git a/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockIterator.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockIterator.scala similarity index 100% rename from src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockIterator.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockIterator.scala diff --git a/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockResolver.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockResolver.scala similarity index 100% rename from src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockResolver.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockResolver.scala diff --git a/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleConf.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleConf.scala similarity index 100% rename from src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleConf.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleConf.scala diff --git a/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleManager.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleManager.scala similarity index 100% rename from src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleManager.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleManager.scala diff --git a/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleReader.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleReader.scala similarity index 100% rename from src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleReader.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleReader.scala diff --git a/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleUtils.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleUtils.scala similarity index 100% rename from src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleUtils.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleUtils.scala diff --git a/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleWriter.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleWriter.scala similarity index 100% rename from src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleWriter.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/shuffle/remote/RemoteShuffleWriter.scala diff --git a/src/main/scala/org/apache/spark/util/collection/RPartitionedAppendOnlyMap.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/util/collection/RPartitionedAppendOnlyMap.scala similarity index 100% rename from src/main/scala/org/apache/spark/util/collection/RPartitionedAppendOnlyMap.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/util/collection/RPartitionedAppendOnlyMap.scala diff --git a/src/main/scala/org/apache/spark/util/collection/RPartitionedPairBuffer.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/util/collection/RPartitionedPairBuffer.scala similarity index 100% rename from src/main/scala/org/apache/spark/util/collection/RPartitionedPairBuffer.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/util/collection/RPartitionedPairBuffer.scala diff --git a/src/main/scala/org/apache/spark/util/collection/RWritablePartitionedPairCollection.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/util/collection/RWritablePartitionedPairCollection.scala similarity index 100% rename from src/main/scala/org/apache/spark/util/collection/RWritablePartitionedPairCollection.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/util/collection/RWritablePartitionedPairCollection.scala diff --git a/src/main/scala/org/apache/spark/util/collection/RemoteAppendOnlyMap.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/util/collection/RemoteAppendOnlyMap.scala similarity index 100% rename from src/main/scala/org/apache/spark/util/collection/RemoteAppendOnlyMap.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/util/collection/RemoteAppendOnlyMap.scala diff --git a/src/main/scala/org/apache/spark/util/collection/RemoteSorter.scala b/shuffle-hadoop/src/main/scala/org/apache/spark/util/collection/RemoteSorter.scala similarity index 100% rename from src/main/scala/org/apache/spark/util/collection/RemoteSorter.scala rename to shuffle-hadoop/src/main/scala/org/apache/spark/util/collection/RemoteSorter.scala diff --git a/src/test/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleWriterSuite.java b/shuffle-hadoop/src/test/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleWriterSuite.java similarity index 100% rename from src/test/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleWriterSuite.java rename to shuffle-hadoop/src/test/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleWriterSuite.java diff --git a/src/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockIteratorSuite.scala b/shuffle-hadoop/src/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockIteratorSuite.scala similarity index 100% rename from src/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockIteratorSuite.scala rename to shuffle-hadoop/src/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockIteratorSuite.scala diff --git a/src/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockObjectWriterSuite.scala b/shuffle-hadoop/src/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockObjectWriterSuite.scala similarity index 100% rename from src/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockObjectWriterSuite.scala rename to shuffle-hadoop/src/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockObjectWriterSuite.scala diff --git a/src/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockResolverSuite.scala b/shuffle-hadoop/src/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockResolverSuite.scala similarity index 100% rename from src/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockResolverSuite.scala rename to shuffle-hadoop/src/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleBlockResolverSuite.scala diff --git a/src/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleManagerSuite.scala b/shuffle-hadoop/src/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleManagerSuite.scala similarity index 100% rename from src/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleManagerSuite.scala rename to shuffle-hadoop/src/test/scala/org/apache/spark/shuffle/remote/RemoteShuffleManagerSuite.scala diff --git a/src/test/scala/org/apache/spark/shuffle/remote/package.scala b/shuffle-hadoop/src/test/scala/org/apache/spark/shuffle/remote/package.scala similarity index 100% rename from src/test/scala/org/apache/spark/shuffle/remote/package.scala rename to shuffle-hadoop/src/test/scala/org/apache/spark/shuffle/remote/package.scala diff --git a/src/test/scala/org/apache/spark/util/collection/RemoteAppendOnlyMapSuite.scala b/shuffle-hadoop/src/test/scala/org/apache/spark/util/collection/RemoteAppendOnlyMapSuite.scala similarity index 100% rename from src/test/scala/org/apache/spark/util/collection/RemoteAppendOnlyMapSuite.scala rename to shuffle-hadoop/src/test/scala/org/apache/spark/util/collection/RemoteAppendOnlyMapSuite.scala diff --git a/src/test/scala/org/apache/spark/util/collection/RemoteSorterSuite.scala b/shuffle-hadoop/src/test/scala/org/apache/spark/util/collection/RemoteSorterSuite.scala similarity index 100% rename from src/test/scala/org/apache/spark/util/collection/RemoteSorterSuite.scala rename to shuffle-hadoop/src/test/scala/org/apache/spark/util/collection/RemoteSorterSuite.scala diff --git a/src/test/scala/org/apache/spark/util/collection/package.scala b/shuffle-hadoop/src/test/scala/org/apache/spark/util/collection/package.scala similarity index 100% rename from src/test/scala/org/apache/spark/util/collection/package.scala rename to shuffle-hadoop/src/test/scala/org/apache/spark/util/collection/package.scala From 16dd495298247519f0b5aa246e92481822eb74b4 Mon Sep 17 00:00:00 2001 From: jiafu zhang Date: Mon, 25 Jan 2021 11:06:15 +0800 Subject: [PATCH 02/12] corrected location of scalastyle-config Signed-off-by: jiafu zhang --- shuffle-daos/pom.xml | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/shuffle-daos/pom.xml b/shuffle-daos/pom.xml index d357694d..73db62a7 100644 --- a/shuffle-daos/pom.xml +++ b/shuffle-daos/pom.xml @@ -10,6 +10,8 @@ 1.1.0 shuffle-daos + OAP Remote Shuffle Based on DAOS Object API + jar @@ -171,8 +173,8 @@ src/test/scala - scalastyle-config.xml - target/scalastyle-output.xml + ${basedir}/scalastyle-config.xml + ${basedir}/target/scalastyle-output.xml UTF-8 UTF-8 From dbc19df8e7e6d91702cbf91140cda85ac99900d0 Mon Sep 17 00:00:00 2001 From: jiafu zhang Date: Wed, 27 Jan 2021 13:35:03 +0000 Subject: [PATCH 03/12] use daos-java version of 1.1.4 Signed-off-by: jiafu zhang --- shuffle-daos/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/shuffle-daos/pom.xml b/shuffle-daos/pom.xml index 73db62a7..91588e6f 100644 --- a/shuffle-daos/pom.xml +++ b/shuffle-daos/pom.xml @@ -202,7 +202,7 @@ io.daos daos-java - 1.1.0-SNAPSHOT + 1.1.4 junit From e430f28fd3f1390d9ebfb2c9f571dd984c2502b5 Mon Sep 17 00:00:00 2001 From: jiafu zhang Date: Mon, 1 Feb 2021 02:07:47 +0000 Subject: [PATCH 04/12] refactor shuffle-daos by abstracting shuffle IO for supporting both synchronous and asynchronous DAOS Object API Signed-off-by: jiafu zhang --- .../apache/spark/shuffle/daos/DaosReader.java | 290 ++++---- .../spark/shuffle/daos/DaosReaderSync.java | 657 ++++++++++++++++++ .../spark/shuffle/daos/DaosShuffleIO.java | 146 +--- .../shuffle/daos/DaosShuffleInputStream.java | 517 +------------- .../apache/spark/shuffle/daos/DaosWriter.java | 577 +-------------- .../spark/shuffle/daos/DaosWriterSync.java | 559 +++++++++++++++ .../apache/spark/shuffle/daos/IOManager.java | 88 +++ .../spark/shuffle/daos/IOManagerAsync.java | 29 + .../spark/shuffle/daos/IOManagerSync.java | 113 +++ .../shuffle/daos/MapPartitionsWriter.scala | 2 - .../apache/spark/shuffle/daos/package.scala | 15 +- .../spark/shuffle/daos/DaosShuffleIOTest.java | 14 +- .../daos/DaosShuffleInputStreamTest.java | 10 +- .../spark/shuffle/daos/DaosWriterTest.java | 21 +- .../shuffle/daos/DaosShuffleReaderSuite.scala | 15 +- 15 files changed, 1682 insertions(+), 1371 deletions(-) create mode 100644 shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosReaderSync.java create mode 100644 shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosWriterSync.java create mode 100644 shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/IOManager.java create mode 100644 shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/IOManagerAsync.java create mode 100644 shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/IOManagerSync.java diff --git a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosReader.java b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosReader.java index fde2ac56..20a48bff 100644 --- a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosReader.java +++ b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosReader.java @@ -24,221 +24,185 @@ package org.apache.spark.shuffle.daos; import io.daos.obj.DaosObject; -import io.daos.obj.IODataDesc; -import io.netty.util.internal.ObjectPool; +import io.netty.buffer.ByteBuf; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkEnv; +import org.apache.spark.shuffle.ShuffleReadMetricsReporter; +import org.apache.spark.storage.BlockId; +import org.apache.spark.storage.BlockManagerId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import scala.Tuple2; +import scala.Tuple3; +import java.io.IOException; +import java.util.LinkedHashMap; import java.util.Map; -import java.util.concurrent.*; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; /** - * A class with {@link DaosObject} wrapped to read data from DAOS in either caller's thread or - * dedicated executor thread. The actual read is performed by {@link DaosObject#fetch(IODataDesc)}. + * A abstract class with {@link DaosObject} wrapped to read data from DAOS. */ -public class DaosReader { +public interface DaosReader { - private DaosObject object; - private Map bufferSourceMap = new ConcurrentHashMap<>(); - - private BoundThreadExecutors executors; - - private Map readerMap; - - private static Logger logger = LoggerFactory.getLogger(DaosReader.class); + DaosObject getObject(); /** - * construct DaosReader with object and dedicated read executors. + * release resources bound with this reader. * - * @param object - * opened DaosObject - * @param executors - * null means read in caller's thread. Submit {@link ReadTask} to dedicate executor retrieved by - * {@link #nextReaderExecutor()} otherwise. + * @param force + * force close even if there is on-going read */ - public DaosReader(DaosObject object, BoundThreadExecutors executors) { - this.object = object; - this.executors = executors; - } - - public DaosObject getObject() { - return object; - } - - public boolean hasExecutors() { - return executors != null; - } + void close(boolean force); /** - * next executor. null if there is no executors being set. + * set global readMap and hook this reader for releasing resources. * - * @return shareable executor instance. null means no executor set. + * @param readerMap + * global reader map */ - public BoundThreadExecutors.SingleThreadExecutor nextReaderExecutor() { - if (executors != null) { - return executors.nextExecutor(); - } - return null; - } + void setReaderMap(Map readerMap); /** - * release resources of all {@link org.apache.spark.shuffle.daos.DaosShuffleInputStream.BufferSource} - * bound with this reader. + * prepare read with some parameters. + * + * @param partSizeMap + * @param maxBytesInFlight + * how many bytes can be read concurrently + * @param maxReqSizeShuffleToMem + * maximum data can be put in memory + * @param metrics + * @return */ - public void close() { - // force releasing - bufferSourceMap.forEach((k, v) -> k.cleanup(true)); - bufferSourceMap.clear(); - if (readerMap != null) { - readerMap.remove(this); - readerMap = null; - } - } - - @Override - public String toString() { - return "DaosReader{" + - "object=" + object + - '}'; - } + void prepare(LinkedHashMap, Tuple3> partSizeMap, + long maxBytesInFlight, long maxReqSizeShuffleToMem, ShuffleReadMetricsReporter metrics); /** - * register buffer source for resource cleanup. + * current map/reduce id being requested. * - * @param source - * BufferSource instance + * @return map/reduce id tuple */ - public void register(DaosShuffleInputStream.BufferSource source) { - bufferSourceMap.put(source, 1); - } + Tuple2 curMapReduceId(); /** - * unregister buffer source if source is release already. + * get available buffer after iterating current buffer, next buffer in current desc and next desc. * - * @param source - * BufferSource instance + * @return buffer with data read from DAOS + * @throws IOException */ - public void unregister(DaosShuffleInputStream.BufferSource source) { - bufferSourceMap.remove(source); - } + ByteBuf nextBuf() throws IOException; /** - * set global readMap and hook this reader for releasing resources. + * All data from current map output is read and + * reach to data from next map? * - * @param readerMap - * global reader map + * @return true or false */ - public void setReaderMap(Map readerMap) { - readerMap.put(this, 0); - this.readerMap = readerMap; - } + boolean isNextMap(); /** - * Task to read from DAOS. Task itself is cached to reduce GC time. - * To reuse task for different reads, prepare and reset {@link ReadTaskContext} by calling - * {@link #newInstance(ReadTaskContext)} + * upper layer should call this method to read more map output */ - static final class ReadTask implements Runnable { - private ReadTaskContext context; - private final ObjectPool.Handle handle; + void setNextMap(boolean b); - private static final ObjectPool objectPool = ObjectPool.newPool(handle -> new ReadTask(handle)); + /** + * check if all data from current map output is read. + */ + void checkPartitionSize() throws IOException; - private static final Logger log = LoggerFactory.getLogger(ReadTask.class); + /** + * check if all map outputs are read. + * + * @throws IOException + */ + void checkTotalPartitions() throws IOException; - static ReadTask newInstance(ReadTaskContext context) { - ReadTask task = objectPool.get(); - task.context = context; - return task; + /** + * reader configurations, please check configs prefixed with SHUFFLE_DAOS_READ in {@link package$#MODULE$}. + */ + final class ReaderConfig { + private long minReadSize; + private long maxBytesInFlight; + private long maxMem; + private int readBatchSize; + private int waitDataTimeMs; + private int waitTimeoutTimes; + private boolean fromOtherThread; + + private static final Logger log = LoggerFactory.getLogger(ReaderConfig.class); + + public ReaderConfig() { + this(true); } - private ReadTask(ObjectPool.Handle handle) { - this.handle = handle; + private ReaderConfig(boolean load) { + if (load) { + initialize(); + } } - @Override - public void run() { - boolean cancelled = context.cancelled; - try { - if (!cancelled) { - context.object.fetch(context.desc); - } - } catch (Exception e) { - log.error("failed to read for " + context.desc, e); - } finally { - // release desc buffer and keep data buffer - context.desc.release(cancelled); - context.signal(); - context = null; - handle.recycle(this); + private void initialize() { + SparkConf conf = SparkEnv.get().conf(); + minReadSize = (long)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_MINIMUM_SIZE()) * 1024; + this.maxBytesInFlight = -1L; + this.maxMem = -1L; + this.readBatchSize = (int)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_BATCH_SIZE()); + this.waitDataTimeMs = (int)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_WAIT_DATA_MS()); + this.waitTimeoutTimes = (int)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_WAIT_DATA_TIMEOUT_TIMES()); + this.fromOtherThread = (boolean)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_FROM_OTHER_THREAD()); + if (log.isDebugEnabled()) { + log.debug("minReadSize: " + minReadSize); + log.debug("maxBytesInFlight: " + maxBytesInFlight); + log.debug("maxMem: " + maxMem); + log.debug("readBatchSize: " + readBatchSize); + log.debug("waitDataTimeMs: " + waitDataTimeMs); + log.debug("waitTimeoutTimes: " + waitTimeoutTimes); + log.debug("fromOtherThread: " + fromOtherThread); } } - } - /** - * Context for read task. It holds all other object to read and sync between caller thread and read thread. - * It should be cached in caller thread for reusing. - */ - static final class ReadTaskContext extends LinkedTaskContext { - - /** - * constructor with all parameters. Some of them can be reused later. - * - * @param object - * DAOS object to fetch data from DAOS - * @param counter - * counter to indicate how many data ready for being consumed - * @param takeLock - * lock to work with notEmpty condition to signal caller thread there is data ready to be consumed - * @param notEmpty - * condition to signal there is some data ready - * @param desc - * desc object to describe which part of data to be fetch and hold returned data - * @param mapReduceId - * to track which map reduce ID this task fetches data for - */ - ReadTaskContext(DaosObject object, AtomicInteger counter, Lock takeLock, Condition notEmpty, - IODataDesc desc, Object mapReduceId) { - super(object, counter, takeLock, notEmpty); - this.desc = desc; - this.morePara = mapReduceId; + public ReaderConfig copy(long maxBytesInFlight, long maxMem) { + ReaderConfig rc = new ReaderConfig(false); + rc.maxMem = maxMem; + rc.minReadSize = minReadSize; + rc.readBatchSize = readBatchSize; + rc.waitDataTimeMs = waitDataTimeMs; + rc.waitTimeoutTimes = waitTimeoutTimes; + rc.fromOtherThread = fromOtherThread; + if (maxBytesInFlight < rc.minReadSize) { + rc.maxBytesInFlight = minReadSize; + } else { + rc.maxBytesInFlight = maxBytesInFlight; + } + return rc; } - @Override - public ReadTaskContext getNext() { - return (ReadTaskContext) next; + public int getReadBatchSize() { + return readBatchSize; } - public Tuple2 getMapReduceId() { - return (Tuple2) morePara; + public int getWaitDataTimeMs() { + return waitDataTimeMs; } - } - /** - * Thread factory for DAOS read tasks. - */ - protected static class ReadThreadFactory implements ThreadFactory { - private AtomicInteger id = new AtomicInteger(0); - - @Override - public Thread newThread(Runnable runnable) { - Thread t; - String name = "daos_read_" + id.getAndIncrement(); - if (runnable == null) { - t = new Thread(name); - } else { - t = new Thread(runnable, name); - } - t.setDaemon(true); - t.setUncaughtExceptionHandler((thread, throwable) -> - logger.error("exception occurred in thread " + name, throwable)); - return t; + public int getWaitTimeoutTimes() { + return waitTimeoutTimes; + } + + public long getMaxBytesInFlight() { + return maxBytesInFlight; } - } + public long getMaxMem() { + return maxMem; + } + + public long getMinReadSize() { + return minReadSize; + } + + public boolean isFromOtherThread() { + return fromOtherThread; + } + } } diff --git a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosReaderSync.java b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosReaderSync.java new file mode 100644 index 00000000..19974540 --- /dev/null +++ b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosReaderSync.java @@ -0,0 +1,657 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos; + +import io.daos.obj.DaosObject; +import io.daos.obj.IODataDesc; +import io.netty.buffer.ByteBuf; +import io.netty.util.internal.ObjectPool; +import org.apache.spark.shuffle.ShuffleReadMetricsReporter; +import org.apache.spark.storage.BlockId; +import org.apache.spark.storage.BlockManagerId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Tuple2; +import scala.Tuple3; + +import java.io.IOException; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; + +/** + * A class with {@link DaosObject} wrapped to read data from DAOS in either caller's thread or + * dedicated executor thread. The actual read is performed by {@link DaosObject#fetch(IODataDesc)}. + * + * User just calls {@link #nextBuf()} and reads from buffer repeatedly until no buffer returned. + * Reader determines when and how (caller thread or from dedicated thread) based on config, to read from DAOS + * as well as controlling buffer size and task batch size. It also has some fault tolerance ability, like + * re-read from caller thread if task doesn't respond from the dedicated threads. + */ +public class DaosReaderSync extends TaskSubmitter implements DaosReader { + + private DaosObject object; + + private Map readerMap; + + private ReaderConfig config; + + protected LinkedHashMap, Tuple3> partSizeMap; + + private Iterator> mapIdIt; + + private ShuffleReadMetricsReporter metrics; + + protected long currentPartSize; + + protected Tuple2 curMapReduceId; + protected Tuple2 lastMapReduceIdForSubmit; + protected Tuple2 lastMapReduceIdForReturn; + protected int curOffset; + protected boolean nextMap; + + protected int totalParts; + protected int partsRead; + + private ReadTaskContext selfCurrentCtx; + private IODataDesc currentDesc; + private IODataDesc.Entry currentEntry; + + private boolean fromOtherThread; + + private int entryIdx; + + private static Logger logger = LoggerFactory.getLogger(DaosReader.class); + + /** + * construct DaosReader with object and dedicated read executors. + * + * @param object + * opened DaosObject + * @param config + * reader configuration + * @param executor + * single thread executor + */ + public DaosReaderSync(DaosObject object, ReaderConfig config, BoundThreadExecutors.SingleThreadExecutor executor) { + super(executor); + this.object = object; + this.config = config; + this.fromOtherThread = config.isFromOtherThread(); + if (fromOtherThread && executor == null) { + throw new IllegalArgumentException("executor should not be null if read from other thread"); + } + } + + @Override + public DaosObject getObject() { + return object; + } + + @Override + public void close(boolean force) { + boolean allReleased = true; + allReleased &= cleanupSubmitted(force); + allReleased &= cleanupConsumed(force); + if (allReleased) { + if (readerMap != null) { + readerMap.remove(this); + readerMap = null; + } + } + } + + @Override + public void setReaderMap(Map readerMap) { + readerMap.put(this, 0); + this.readerMap = readerMap; + } + + public boolean hasExecutors() { + return executor != null; + } + + /** + * invoke this method when fromOtherThread is false. + * + * @return + * @throws {@link IOException} + */ + public ByteBuf readBySelf() throws IOException { + if (lastCtx != null) { // duplicated IODataDescs which were submitted to other thread, but cancelled + ByteBuf buf = readDuplicated(false); + if (buf != null) { + return buf; + } + } + // all submitted were duplicated. Now start from mapId iterator. + IODataDesc desc = createNextDesc(config.getMaxBytesInFlight()); + return getBySelf(desc, lastMapReduceIdForSubmit); + } + + /** + * get available buffer after iterating current buffer, next buffer in current desc and next desc. + * + * @return buffer with data read from DAOS + * @throws IOException + */ + public ByteBuf nextBuf() throws IOException { + ByteBuf buf = tryCurrentEntry(); + if (buf != null) { + return buf; + } + // next entry + buf = tryCurrentDesc(); + if (buf != null) { + return buf; + } + // from next partition + if (fromOtherThread) { + // next ready queue + if (headCtx != null) { + return tryNextTaskContext(); + } + // get data by self and submit request for remaining data + return getBySelfAndSubmitMore(config.getMinReadSize()); + } + // get data by self after fromOtherThread disabled + return readBySelf(); + } + + @Override + public boolean isNextMap() { + return nextMap; + } + + @Override + public void setNextMap(boolean nextMap) { + this.nextMap = nextMap; + } + + private ByteBuf tryNextTaskContext() throws IOException { + // make sure there are still some read tasks waiting/running/returned from other threads + // or they are readDuplicated by self + if (totalSubmitted == 0 || selfCurrentCtx == lastCtx) { + return getBySelfAndSubmitMore(config.getMaxBytesInFlight()); + } + if (totalSubmitted < 0) { + throw new IllegalStateException("total submitted should be no less than 0. " + totalSubmitted); + } + try { + IODataDesc desc; + if ((desc = tryGetFromOtherThread()) != null) { + submitMore(); + return validateLastEntryAndGetBuf(desc.getEntry(entryIdx)); + } + // duplicate and get data by self + return readDuplicated(true); + } catch (InterruptedException e) { + throw new IOException("read interrupted.", e); + } + } + + /** + * we have to duplicate submitted desc since mapId was moved. + * + * @return + * @throws IOException + */ + private ByteBuf readDuplicated(boolean expectNotNullCtx) throws IOException { + ReadTaskContext context = getNextNonReturnedCtx(); + if (context == null) { + if (expectNotNullCtx) { + throw new IllegalStateException("context should not be null. totalSubmitted: " + totalSubmitted); + } + if (!fromOtherThread) { + lastCtx = null; + } + return null; + } + IODataDesc newDesc = context.getDesc().duplicate(); + ByteBuf buf = getBySelf(newDesc, context.getMapReduceId()); + selfCurrentCtx = context; + return buf; + } + + private IODataDesc tryGetFromOtherThread() throws InterruptedException, IOException { + IODataDesc desc = tryGetValidCompleted(); + if (desc != null) { + return desc; + } + // check completion + if ((!mapIdIt.hasNext()) && curMapReduceId == null && totalSubmitted == 0) { + return null; + } + // wait for specified time + desc = waitForValidFromOtherThread(); + if (desc != null) { + return desc; + } + // check wait times and cancel task + // TODO: stop reading from other threads? + cancelTasks(false); + return null; + } + + private IODataDesc waitForValidFromOtherThread() throws InterruptedException, IOException { + IODataDesc desc; + while (true) { + long start = System.nanoTime(); + boolean timeout = waitForCondition(config.getWaitDataTimeMs()); + metrics.incFetchWaitTime(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start)); + if (timeout) { + exceedWaitTimes++; + if (logger.isDebugEnabled()) { + logger.debug("exceed wait: {}ms, times: {}", config.getWaitDataTimeMs(), exceedWaitTimes); + } + if (exceedWaitTimes >= config.getWaitTimeoutTimes()) { + return null; + } + } + // get some results after wait + desc = tryGetValidCompleted(); + if (desc != null) { + return desc; + } + } + } + + protected IODataDesc tryGetValidCompleted() throws IOException { + if (moveForward()) { + return currentDesc; + } + return null; + } + + private ByteBuf tryCurrentDesc() throws IOException { + if (currentDesc != null) { + ByteBuf buf; + while (entryIdx < currentDesc.getNbrOfEntries()) { + IODataDesc.Entry entry = currentDesc.getEntry(entryIdx); + buf = validateLastEntryAndGetBuf(entry); + if (buf.readableBytes() > 0) { + return buf; + } + entryIdx++; + } + entryIdx = 0; + // no need to release desc since all its entries are released in tryCurrentEntry and + // internal buffers are released after object.fetch + // reader.close will release all in case of failure + currentDesc = null; + } + return null; + } + + private ByteBuf tryCurrentEntry() { + if (currentEntry != null && !currentEntry.isFetchBufReleased()) { + ByteBuf buf = currentEntry.getFetchedData(); + if (buf.readableBytes() > 0) { + return buf; + } + // release buffer as soon as possible + currentEntry.releaseDataBuffer(); + entryIdx++; + } + // not null currentEntry since it will be used for size validation + return null; + } + + /** + * for first read. + * + * @param selfReadLimit + * @return + * @throws IOException + */ + private ByteBuf getBySelfAndSubmitMore(long selfReadLimit) throws IOException { + entryIdx = 0; + // fetch the next by self + IODataDesc desc = createNextDesc(selfReadLimit); + Tuple2 mapreduceId = lastMapReduceIdForSubmit; + try { + if (fromOtherThread) { + submitMore(); + } + } catch (Exception e) { + desc.release(); + if (e instanceof IOException) { + throw (IOException)e; + } + throw new IOException("failed to submit more", e); + } + // first time read from reduce task + return getBySelf(desc, mapreduceId); + } + + private void submitMore() throws IOException { + while (totalSubmitted < config.getReadBatchSize() && totalInMemSize < config.getMaxMem()) { + IODataDesc taskDesc = createNextDesc(config.getMaxBytesInFlight()); + if (taskDesc == null) { + break; + } + submit(taskDesc, lastMapReduceIdForSubmit); + } + } + + private ByteBuf getBySelf(IODataDesc desc, Tuple2 mapreduceId) throws IOException { + // get data by self, no need to release currentDesc + if (desc == null) { // reach end + return null; + } + boolean releaseBuf = false; + try { + object.fetch(desc); + currentDesc = desc; + ByteBuf buf = validateLastEntryAndGetBuf(desc.getEntry(entryIdx)); + lastMapReduceIdForReturn = mapreduceId; + return buf; + } catch (IOException | IllegalStateException e) { + releaseBuf = true; + throw e; + } finally { + desc.release(releaseBuf); + } + } + + private IODataDesc createNextDesc(long sizeLimit) throws IOException { + long remaining = sizeLimit; + int reduceId = -1; + long mapId; + IODataDesc desc = null; + while (remaining > 0) { + nextMapReduceId(); + if (curMapReduceId == null) { + break; + } + if (reduceId > 0 && curMapReduceId._2 != reduceId) { // make sure entries under same reduce + break; + } + reduceId = curMapReduceId._2; + mapId = curMapReduceId._1; + lastMapReduceIdForSubmit = curMapReduceId; + long readSize = partSizeMap.get(curMapReduceId)._1() - curOffset; + long offset = curOffset; + if (readSize > remaining) { + readSize = remaining; + curOffset += readSize; + } else { + curOffset = 0; + curMapReduceId = null; + } + if (desc == null) { + desc = object.createDataDescForFetch(String.valueOf(reduceId), IODataDesc.IodType.ARRAY, 1); + } + desc.addEntryForFetch(String.valueOf(mapId), (int)offset, (int)readSize); + remaining -= readSize; + } + return desc; + } + + private void nextMapReduceId() { + if (curMapReduceId != null) { + return; + } + curOffset = 0; + if (mapIdIt.hasNext()) { + curMapReduceId = mapIdIt.next(); + partsRead++; + } else { + curMapReduceId = null; + } + } + + private ByteBuf validateLastEntryAndGetBuf(IODataDesc.Entry entry) throws IOException { + ByteBuf buf = entry.getFetchedData(); + int byteLen = buf.readableBytes(); + nextMap = false; + if (currentEntry != null && entry != currentEntry) { + if (entry.getKey().equals(currentEntry.getKey())) { + currentPartSize += byteLen; + } else { + checkPartitionSize(); + nextMap = true; + currentPartSize = byteLen; + } + } + currentEntry = entry; + metrics.incRemoteBytesRead(byteLen); + return buf; + } + + @Override + public void checkPartitionSize() throws IOException { + if (lastMapReduceIdForReturn == null) { + return; + } + // partition size is not accurate after compress/decompress + long size = partSizeMap.get(lastMapReduceIdForReturn)._1(); + if (size < 35 * 1024 * 1024 * 1024 && currentPartSize * 1.1 < size) { + throw new IOException("expect partition size " + partSizeMap.get(lastMapReduceIdForReturn) + + ", actual size " + currentPartSize + ", mapId and reduceId: " + lastMapReduceIdForReturn); + } + metrics.incRemoteBlocksFetched(1); + } + + @Override + public void checkTotalPartitions() throws IOException { + if (partsRead != totalParts) { + throw new IOException("expect total partitions to be read: " + totalParts + ", actual read: " + partsRead); + } + } + + @Override + public void prepare(LinkedHashMap, Tuple3> partSizeMap, + long maxBytesInFlight, long maxReqSizeShuffleToMem, + ShuffleReadMetricsReporter metrics) { + this.partSizeMap = partSizeMap; + this.config = config.copy(maxBytesInFlight, maxReqSizeShuffleToMem); + this.metrics = metrics; + this.totalParts = partSizeMap.size(); + mapIdIt = partSizeMap.keySet().iterator(); + } + + @Override + public Tuple2 curMapReduceId() { + return lastMapReduceIdForSubmit; + } + + @Override + protected ReadTaskContext getNextNonReturnedCtx() { + // in case no even single return from other thread + // check selfCurrentCtx since the wait could span multiple contexts/descs + ReadTaskContext curCtx = selfCurrentCtx == null ? + getCurrentCtx() : selfCurrentCtx; + if (curCtx == null) { + return getHeadCtx(); + } + // no consumedStack push and no totalInMemSize and totalSubmitted update + // since they will be updated when the task context finally returned + return curCtx.getNext(); + } + + @Override + protected boolean consumed(LinkedTaskContext consumed) { + return !consumed.isCancelled(); + } + + @Override + protected boolean validateReturned(LinkedTaskContext context) throws IOException { + if (context.isCancelled()) { + return false; + } + selfCurrentCtx = null; // non-cancelled currentCtx overrides selfCurrentCtx + lastMapReduceIdForReturn = ((ReadTaskContext)context).getMapReduceId(); + IODataDesc desc = context.getDesc(); + if (!desc.isSucceeded()) { + String msg = "failed to get data from DAOS, desc: " + desc.toString(4096); + if (desc.getCause() != null) { + throw new IOException(msg, desc.getCause()); + } else { + throw new IllegalStateException(msg + "\nno exception got. logic error or crash?"); + } + } + currentDesc = desc; + return true; + } + + @Override + protected Runnable newTask(LinkedTaskContext context) { + return ReadTask.newInstance((ReadTaskContext) context); + } + + @Override + protected LinkedTaskContext createTaskContext(IODataDesc desc, Object morePara) { + return new ReadTaskContext(object, counter, lock, condition, desc, morePara); + } + + @Override + public ReadTaskContext getCurrentCtx() { + return (ReadTaskContext) currentCtx; + } + + @Override + public ReadTaskContext getHeadCtx() { + return (ReadTaskContext) headCtx; + } + + @Override + public ReadTaskContext getLastCtx() { + return (ReadTaskContext) lastCtx; + } + + @Override + public String toString() { + return "DaosReaderSync{" + + "object=" + object + + '}'; + } + + /** + * Task to read from DAOS. Task itself is cached to reduce GC time. + * To reuse task for different reads, prepare and reset {@link ReadTaskContext} by calling + * {@link #newInstance(ReadTaskContext)} + */ + static final class ReadTask implements Runnable { + private ReadTaskContext context; + private final ObjectPool.Handle handle; + + private static final ObjectPool objectPool = ObjectPool.newPool(handle -> new ReadTask(handle)); + + private static final Logger log = LoggerFactory.getLogger(ReadTask.class); + + static ReadTask newInstance(ReadTaskContext context) { + ReadTask task = objectPool.get(); + task.context = context; + return task; + } + + private ReadTask(ObjectPool.Handle handle) { + this.handle = handle; + } + + @Override + public void run() { + boolean cancelled = context.cancelled; + try { + if (!cancelled) { + context.object.fetch(context.desc); + } + } catch (Exception e) { + log.error("failed to read for " + context.desc, e); + } finally { + // release desc buffer and keep data buffer + context.desc.release(cancelled); + context.signal(); + context = null; + handle.recycle(this); + } + } + } + + /** + * Context for read task. It holds all other object to read and sync between caller thread and read thread. + * It should be cached in caller thread for reusing. + */ + static final class ReadTaskContext extends LinkedTaskContext { + + /** + * constructor with all parameters. Some of them can be reused later. + * + * @param object + * DAOS object to fetch data from DAOS + * @param counter + * counter to indicate how many data ready for being consumed + * @param takeLock + * lock to work with notEmpty condition to signal caller thread there is data ready to be consumed + * @param notEmpty + * condition to signal there is some data ready + * @param desc + * desc object to describe which part of data to be fetch and hold returned data + * @param mapReduceId + * to track which map reduce ID this task fetches data for + */ + ReadTaskContext(DaosObject object, AtomicInteger counter, Lock takeLock, Condition notEmpty, + IODataDesc desc, Object mapReduceId) { + super(object, counter, takeLock, notEmpty); + this.desc = desc; + this.morePara = mapReduceId; + } + + @Override + public ReadTaskContext getNext() { + return (ReadTaskContext) next; + } + + public Tuple2 getMapReduceId() { + return (Tuple2) morePara; + } + } + + /** + * Thread factory for DAOS read tasks. + */ + protected static class ReadThreadFactory implements ThreadFactory { + private AtomicInteger id = new AtomicInteger(0); + + @Override + public Thread newThread(Runnable runnable) { + Thread t; + String name = "daos_read_" + id.getAndIncrement(); + if (runnable == null) { + t = new Thread(name); + } else { + t = new Thread(runnable, name); + } + t.setDaemon(true); + t.setUncaughtExceptionHandler((thread, throwable) -> + logger.error("exception occurred in thread " + name, throwable)); + return t; + } + } +} diff --git a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleIO.java b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleIO.java index 26990085..dd0aea90 100644 --- a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleIO.java +++ b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleIO.java @@ -25,10 +25,7 @@ import io.daos.obj.DaosObjClient; import io.daos.obj.DaosObject; -import io.daos.obj.DaosObjectException; -import io.daos.obj.DaosObjectId; import org.apache.spark.SparkConf; -import org.apache.spark.launcher.SparkLauncher; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,6 +47,8 @@ public class DaosShuffleIO { private DaosObjClient objClient; + private Map objectMap = new ConcurrentHashMap<>(); + private SparkConf conf; private Map driverConf; @@ -58,21 +57,9 @@ public class DaosShuffleIO { private String contId; - private String ranks; - private boolean removeShuffleData; - private DaosWriter.WriteConfig writeConfig; - - private Map readerMap = new ConcurrentHashMap<>(); - - private Map writerMap = new ConcurrentHashMap<>(); - - private Map objectMap = new ConcurrentHashMap<>(); - - private BoundThreadExecutors readerExes; - - private BoundThreadExecutors writerExes; + private IOManager ioManager; private static final Logger logger = LoggerFactory.getLogger(DaosShuffleIO.class); @@ -84,61 +71,11 @@ public class DaosShuffleIO { */ public DaosShuffleIO(SparkConf conf) { this.conf = conf; - this.writeConfig = loadWriteConfig(conf); - this.readerExes = createReaderExes(); - this.writerExes = createWriterExes(); + boolean async = (boolean)conf.get(package$.MODULE$.SHUFFLE_DAOS_IO_ASYNC()); + this.ioManager = async ? new IOManagerAsync(conf, objectMap) : new IOManagerSync(conf, objectMap); this.removeShuffleData = (boolean)conf.get(package$.MODULE$.SHUFFLE_DAOS_REMOVE_SHUFFLE_DATA()); } - protected static DaosWriter.WriteConfig loadWriteConfig(SparkConf conf) { - DaosWriter.WriteConfig config = new DaosWriter.WriteConfig(); - config.warnSmallWrite((boolean)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_WARN_SMALL_SIZE())); - config.bufferSize((int) ((long)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_SINGLE_BUFFER_SIZE()) - * 1024 * 1024)); - config.minSize((int) ((long)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_MINIMUM_SIZE()) * 1024)); - config.timeoutTimes((int)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_WAIT_DATA_TIMEOUT_TIMES())); - config.waitTimeMs((int)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_WAIT_MS())); - config.totalInMemSize((long)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_MAX_BYTES_IN_FLIGHT()) * 1024); - config.totalSubmittedLimit((int)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_SUBMITTED_LIMIT())); - config.threads((int)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_THREADS())); - config.fromOtherThreads((boolean)conf - .get(package$.MODULE$.SHUFFLE_DAOS_WRITE_IN_OTHER_THREAD())); - logger.info("write configs, " + config); - return config; - } - - private BoundThreadExecutors createWriterExes() { - if (writeConfig.isFromOtherThreads()) { - BoundThreadExecutors executors; - int threads = writeConfig.getThreads(); - if (threads == -1) { - threads = conf.getInt(SparkLauncher.EXECUTOR_CORES, 1); - } - executors = new BoundThreadExecutors("write_executors", threads, - new DaosWriter.WriteThreadFactory()); - logger.info("created BoundThreadExecutors with " + threads + " threads for write"); - return executors; - } - return null; - } - - private BoundThreadExecutors createReaderExes() { - boolean fromOtherThread = (boolean)conf - .get(package$.MODULE$.SHUFFLE_DAOS_READ_FROM_OTHER_THREAD()); - if (fromOtherThread) { - BoundThreadExecutors executors; - int threads = (int)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_THREADS()); - if (threads == -1) { - threads = conf.getInt(SparkLauncher.EXECUTOR_CORES, 1); - } - executors = new BoundThreadExecutors("read_executors", threads, - new DaosReader.ReadThreadFactory()); - logger.info("created BoundThreadExecutors with " + threads + " threads for read"); - return executors; - } - return null; - } - /** * connect DAOS server. * @@ -149,18 +86,14 @@ public void initialize(Map driverConf) throws IOException { this.driverConf = driverConf; poolId = conf.get(package$.MODULE$.SHUFFLE_DAOS_POOL_UUID()); contId = conf.get(package$.MODULE$.SHUFFLE_DAOS_CONTAINER_UUID()); - ranks = conf.get(package$.MODULE$.SHUFFLE_DAOS_POOL_RANKS()); if (poolId == null || contId == null) { throw new IllegalArgumentException("DaosShuffleManager needs pool id and container id"); } objClient = new DaosObjClient.DaosObjClientBuilder() - .poolId(poolId).containerId(contId).ranks(ranks) + .poolId(poolId).containerId(contId) .build(); - } - - private long parseAppId(String appId) { - return Long.valueOf(appId.replaceAll("\\D", "")); + ioManager.setObjClient(objClient); } /** @@ -177,20 +110,7 @@ private long parseAppId(String appId) { */ public DaosWriter getDaosWriter(int numPartitions, int shuffleId, long mapId) throws IOException { - long appId = parseAppId(conf.getAppId()); - if (logger.isDebugEnabled()) { - logger.debug("getting daoswriter for app id: " + appId + ", shuffle id: " + shuffleId + ", map id: " + mapId + - ", numPartitions: " + numPartitions); - } - DaosWriter.WriteParam param = new DaosWriter.WriteParam(); - param.numPartitions(numPartitions) - .shuffleId(shuffleId) - .mapId(mapId) - .config(writeConfig); - DaosWriter writer = new DaosWriter(param, getObject(appId, shuffleId), - writerExes == null ? null : writerExes.nextExecutor()); - writer.setWriterMap(writerMap); - return writer; + return ioManager.getDaosWriter(numPartitions, shuffleId, mapId); } /** @@ -198,43 +118,18 @@ public DaosWriter getDaosWriter(int numPartitions, int shuffleId, long mapId) * * @param shuffleId * @return DaosReader - * @throws DaosObjectException + * @throws IOException */ - public DaosReader getDaosReader(int shuffleId) throws DaosObjectException { - long appId = parseAppId(conf.getAppId()); - if (logger.isDebugEnabled()) { - logger.debug("getting daosreader for app id: " + appId + ", shuffle id: " + shuffleId); - } - DaosReader reader = new DaosReader(getObject(appId, shuffleId), readerExes); - reader.setReaderMap(readerMap); - return reader; + public DaosReader getDaosReader(int shuffleId) throws IOException { + return ioManager.getDaosReader(shuffleId); } private String getKey(long appId, int shuffleId) { return appId + "" + shuffleId; } - private DaosObject getObject(long appId, int shuffleId) throws DaosObjectException { - String key = getKey(appId, shuffleId); - DaosObject object = objectMap.get(key); - if (object == null) { - DaosObjectId id = new DaosObjectId(appId, shuffleId); - id.encode(); - object = objClient.getObject(id); - objectMap.putIfAbsent(key, object); - DaosObject activeObject = objectMap.get(key); - if (activeObject != object) { // release just created DaosObject - object.close(); - object = activeObject; - } - } - // open just once in multiple threads - if (!object.isOpen()) { - synchronized (object) { - object.open(); - } - } - return object; + public IOManager getIoManager() { + return ioManager; } /** @@ -244,7 +139,7 @@ private DaosObject getObject(long appId, int shuffleId) throws DaosObjectExcepti * @return */ public boolean removeShuffle(int shuffleId) { - long appId = parseAppId(conf.getAppId()); + long appId = IOManager.parseAppId(conf.getAppId()); logger.info("punching daos object for app id: " + appId + ", shuffle id: " + shuffleId); try { DaosObject object = objectMap.remove(getKey(appId, shuffleId)); @@ -267,18 +162,7 @@ public boolean removeShuffle(int shuffleId) { * @throws IOException */ public void close() throws IOException { - if (readerExes != null) { - readerExes.stop(); - readerMap.keySet().forEach(r -> r.close()); - readerMap.clear(); - readerExes = null; - } - if (writerExes != null) { - writerExes.stop(); - writerMap.keySet().forEach(r -> r.close()); - writerMap.clear(); - writerExes = null; - } + ioManager.close(); objClient.forceClose(); } } diff --git a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleInputStream.java b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleInputStream.java index 264168a3..a1b5f5c9 100644 --- a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleInputStream.java +++ b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosShuffleInputStream.java @@ -24,10 +24,7 @@ package org.apache.spark.shuffle.daos; import io.daos.obj.DaosObject; -import io.daos.obj.IODataDesc; import io.netty.buffer.ByteBuf; -import org.apache.spark.SparkConf; -import org.apache.spark.SparkEnv; import org.apache.spark.shuffle.ShuffleReadMetricsReporter; import org.apache.spark.storage.BlockId; import org.apache.spark.storage.BlockManagerId; @@ -40,7 +37,6 @@ import java.io.IOException; import java.io.InputStream; import java.util.*; -import java.util.concurrent.*; @NotThreadSafe /** @@ -58,14 +54,8 @@ public class DaosShuffleInputStream extends InputStream { private DaosObject object; - private BoundThreadExecutors.SingleThreadExecutor executor; - - private ReaderConfig config; - private ShuffleReadMetricsReporter metrics; - private boolean fromOtherThread; - private volatile boolean cleaned; private boolean completed; @@ -73,14 +63,11 @@ public class DaosShuffleInputStream extends InputStream { // ensure the order of partition // (mapid, reduceid) -> (length, BlockId, BlockManagerId) private LinkedHashMap, Tuple3> partSizeMap; - private Iterator> mapIdIt; - - private BufferSource source; private static final Logger log = LoggerFactory.getLogger(DaosShuffleInputStream.class); /** - * constructor with ordered map outputs info. Check {@link ReaderConfig} for more paras controlling + * constructor with ordered map outputs info. Check {@link DaosReader.ReaderConfig} for more paras controlling * how data being read from DAOS. * * @param reader @@ -89,7 +76,7 @@ public class DaosShuffleInputStream extends InputStream { * ordered map outputs info. They are organize as (mapid, reduceid) -> (length, BlockId, BlockManagerId) * @param maxBytesInFlight * how many bytes can be read concurrently - * @param maxMem + * @param maxReqSizeShuffleToMem * maximum data can be put in memory * @param metrics * read metrics @@ -97,51 +84,45 @@ public class DaosShuffleInputStream extends InputStream { public DaosShuffleInputStream( DaosReader reader, LinkedHashMap, Tuple3> partSizeMap, - long maxBytesInFlight, long maxMem, ShuffleReadMetricsReporter metrics) { + long maxBytesInFlight, long maxReqSizeShuffleToMem, + ShuffleReadMetricsReporter metrics) { this.partSizeMap = partSizeMap; this.reader = reader; - this.config = new ReaderConfig(maxBytesInFlight, maxMem); - this.fromOtherThread = config.fromOtherThread; - if (fromOtherThread) { - this.executor = reader.nextReaderExecutor(); - } - this.source = new BufferSource(executor); - reader.register(source); + reader.prepare(partSizeMap, maxBytesInFlight, maxReqSizeShuffleToMem, metrics); this.object = reader.getObject(); this.metrics = metrics; - this.mapIdIt = partSizeMap.keySet().iterator(); } public BlockId getCurBlockId() { - if (source.lastMapReduceIdForSubmit == null) { + if (reader.curMapReduceId() == null) { return null; } - return partSizeMap.get(source.lastMapReduceIdForSubmit)._2(); + return partSizeMap.get(reader.curMapReduceId())._2(); } public BlockManagerId getCurOriginAddress() { - if (source.lastMapReduceIdForSubmit == null) { + if (reader.curMapReduceId() == null) { return null; } - return partSizeMap.get(source.lastMapReduceIdForSubmit)._3(); + return partSizeMap.get(reader.curMapReduceId())._3(); } public long getCurMapIndex() { - if (source.lastMapReduceIdForSubmit == null) { + if (reader.curMapReduceId() == null) { return -1; } - return source.lastMapReduceIdForSubmit._1; + return reader.curMapReduceId()._1; } @Override public int read() throws IOException { while (!completed) { - ByteBuf buf = source.nextBuf(); + ByteBuf buf = reader.nextBuf(); if (buf == null) { // reach end complete(); return -1; } - if (source.newMap) { // indication to close upper layer object inputstream + if (reader.isNextMap()) { // indication to close upper layer object inputstream return -1; } if (buf.readableBytes() >= 1) { @@ -160,13 +141,13 @@ public int read(byte[] bytes) throws IOException { public int read(byte[] bytes, int offset, int length) throws IOException { int len = length; while (!completed) { - ByteBuf buf = source.nextBuf(); + ByteBuf buf = reader.nextBuf(); if (buf == null) { // reach end complete(); int r = length - len; return r == 0 ? -1 : r; } - if (source.newMap) { // indication to close upper layer object inputstream + if (reader.isNextMap()) { // indication to close upper layer object inputstream int r = length - len; return r == 0 ? -1 : r; } @@ -186,24 +167,20 @@ public int read(byte[] bytes, int offset, int length) throws IOException { * upper layer should call this method to read more map output */ public void nextMap() { - source.newMap = false; + reader.setNextMap(false); } private void complete() throws IOException { if (!completed) { - source.checkPartitionSize(); - source.checkTotalPartitions(); + reader.checkPartitionSize(); + reader.checkTotalPartitions(); completed = true; } } private void cleanup() { if (!cleaned) { - boolean allReleased = source.cleanup(false); - if (allReleased) { - reader.unregister(source); - } - source = null; + reader.close(false); cleaned = true; completed = true; } @@ -231,460 +208,4 @@ public void close(boolean force) { public boolean isCompleted() { return completed; } - - /** - * Source of map output data. User just calls {@link #nextBuf()} and reads from buffer repeatedly until no buffer - * returned. - * BufferSource does all other dirty things, like when and how (caller thread or from dedicated thread) to - * read from DAOS as well as controlling buffer size and task batch size. - * It also has some fault tolerance ability, like re-read from caller thread if task doesn't respond from the - * dedicated threads. - */ - public class BufferSource extends TaskSubmitter { - private DaosReader.ReadTaskContext selfCurrentCtx; - private IODataDesc currentDesc; - private IODataDesc.Entry currentEntry; - private long currentPartSize; - - private int entryIdx; - private Tuple2 curMapReduceId; - private Tuple2 lastMapReduceIdForSubmit; - private Tuple2 lastMapReduceIdForReturn; - private int curOffset; - private boolean newMap; - - private int totalParts = partSizeMap.size(); - private int partsRead; - - protected BufferSource(BoundThreadExecutors.SingleThreadExecutor executor) { - super(executor); - } - - /** - * invoke this method when fromOtherThread is false. - * - * @return - * @throws {@link IOException} - */ - public ByteBuf readBySelf() throws IOException { - if (lastCtx != null) { // duplicated IODataDescs which were submitted to other thread, but cancelled - ByteBuf buf = readDuplicated(false); - if (buf != null) { - return buf; - } - } - // all submitted were duplicated. Now start from mapId iterator. - IODataDesc desc = createNextDesc(config.maxBytesInFlight); - return getBySelf(desc, lastMapReduceIdForSubmit); - } - - /** - * get available buffer after iterating current buffer, next buffer in current desc and next desc. - * - * @return buffer with data read from DAOS - * @throws IOException - */ - public ByteBuf nextBuf() throws IOException { - ByteBuf buf = tryCurrentEntry(); - if (buf != null) { - return buf; - } - // next entry - buf = tryCurrentDesc(); - if (buf != null) { - return buf; - } - // from next partition - if (fromOtherThread) { - // next ready queue - if (headCtx != null) { - return tryNextTaskContext(); - } - // get data by self and submit request for remaining data - return getBySelfAndSubmitMore(config.minReadSize); - } - // get data by self after fromOtherThread disabled - return readBySelf(); - } - - private ByteBuf tryNextTaskContext() throws IOException { - // make sure there are still some read tasks waiting/running/returned from other threads - // or they are readDuplicated by self - if (totalSubmitted == 0 || selfCurrentCtx == lastCtx) { - return getBySelfAndSubmitMore(config.maxBytesInFlight); - } - if (totalSubmitted < 0) { - throw new IllegalStateException("total submitted should be no less than 0. " + totalSubmitted); - } - try { - IODataDesc desc; - if ((desc = tryGetFromOtherThread()) != null) { - submitMore(); - return validateLastEntryAndGetBuf(desc.getEntry(entryIdx)); - } - // duplicate and get data by self - return readDuplicated(true); - } catch (InterruptedException e) { - throw new IOException("read interrupted.", e); - } - } - - /** - * we have to duplicate submitted desc since mapId was moved. - * - * @return - * @throws IOException - */ - private ByteBuf readDuplicated(boolean expectNotNullCtx) throws IOException { - DaosReader.ReadTaskContext context = getNextNonReturnedCtx(); - if (context == null) { - if (expectNotNullCtx) { - throw new IllegalStateException("context should not be null. totalSubmitted: " + totalSubmitted); - } - if (!fromOtherThread) { - lastCtx = null; - } - return null; - } - IODataDesc newDesc = context.getDesc().duplicate(); - ByteBuf buf = getBySelf(newDesc, context.getMapReduceId()); - selfCurrentCtx = context; - return buf; - } - - @Override - protected DaosReader.ReadTaskContext getNextNonReturnedCtx() { - // in case no even single return from other thread - // check selfCurrentCtx since the wait could span multiple contexts/descs - DaosReader.ReadTaskContext curCtx = selfCurrentCtx == null ? - getCurrentCtx() : selfCurrentCtx; - if (curCtx == null) { - return getHeadCtx(); - } - // no consumedStack push and no totalInMemSize and totalSubmitted update - // since they will be updated when the task context finally returned - return curCtx.getNext(); - } - - private IODataDesc tryGetFromOtherThread() throws InterruptedException, IOException { - IODataDesc desc = tryGetValidCompleted(); - if (desc != null) { - return desc; - } - // check completion - if ((!mapIdIt.hasNext()) && curMapReduceId == null && totalSubmitted == 0) { - return null; - } - // wait for specified time - desc = waitForValidFromOtherThread(); - if (desc != null) { - return desc; - } - // check wait times and cancel task - // TODO: stop reading from other threads? - cancelTasks(false); - return null; - } - - private IODataDesc waitForValidFromOtherThread() throws InterruptedException, IOException { - IODataDesc desc; - while (true) { - long start = System.nanoTime(); - boolean timeout = waitForCondition(config.waitDataTimeMs); - metrics.incFetchWaitTime(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start)); - if (timeout) { - exceedWaitTimes++; - if (log.isDebugEnabled()) { - log.debug("exceed wait: {}ms, times: {}", config.waitDataTimeMs, exceedWaitTimes); - } - if (exceedWaitTimes >= config.waitTimeoutTimes) { - return null; - } - } - // get some results after wait - desc = tryGetValidCompleted(); - if (desc != null) { - return desc; - } - } - } - - protected IODataDesc tryGetValidCompleted() throws IOException { - if (moveForward()) { - return currentDesc; - } - return null; - } - - @Override - protected boolean consumed(LinkedTaskContext consumed) { - return !consumed.isCancelled(); - } - - @Override - protected boolean validateReturned(LinkedTaskContext context) throws IOException { - if (context.isCancelled()) { - return false; - } - selfCurrentCtx = null; // non-cancelled currentCtx overrides selfCurrentCtx - lastMapReduceIdForReturn = ((DaosReader.ReadTaskContext)context).getMapReduceId(); - IODataDesc desc = context.getDesc(); - if (!desc.isSucceeded()) { - String msg = "failed to get data from DAOS, desc: " + desc.toString(4096); - if (desc.getCause() != null) { - throw new IOException(msg, desc.getCause()); - } else { - throw new IllegalStateException(msg + "\nno exception got. logic error or crash?"); - } - } - currentDesc = desc; - return true; - } - - private ByteBuf tryCurrentDesc() throws IOException { - if (currentDesc != null) { - ByteBuf buf; - while (entryIdx < currentDesc.getNbrOfEntries()) { - IODataDesc.Entry entry = currentDesc.getEntry(entryIdx); - buf = validateLastEntryAndGetBuf(entry); - if (buf.readableBytes() > 0) { - return buf; - } - entryIdx++; - } - entryIdx = 0; - // no need to release desc since all its entries are released in tryCurrentEntry and - // internal buffers are released after object.fetch - // reader.close will release all in case of failure - currentDesc = null; - } - return null; - } - - private ByteBuf tryCurrentEntry() { - if (currentEntry != null && !currentEntry.isFetchBufReleased()) { - ByteBuf buf = currentEntry.getFetchedData(); - if (buf.readableBytes() > 0) { - return buf; - } - // release buffer as soon as possible - currentEntry.releaseDataBuffer(); - entryIdx++; - } - // not null currentEntry since it will be used for size validation - return null; - } - - /** - * for first read. - * - * @param selfReadLimit - * @return - * @throws IOException - */ - private ByteBuf getBySelfAndSubmitMore(long selfReadLimit) throws IOException { - entryIdx = 0; - // fetch the next by self - IODataDesc desc = createNextDesc(selfReadLimit); - Tuple2 mapreduceId = lastMapReduceIdForSubmit; - try { - if (fromOtherThread) { - submitMore(); - } - } catch (Exception e) { - desc.release(); - if (e instanceof IOException) { - throw (IOException)e; - } - throw new IOException("failed to submit more", e); - } - // first time read from reduce task - return getBySelf(desc, mapreduceId); - } - - private void submitMore() throws IOException { - while (totalSubmitted < config.readBatchSize && totalInMemSize < config.maxMem) { - IODataDesc taskDesc = createNextDesc(config.maxBytesInFlight); - if (taskDesc == null) { - break; - } - submit(taskDesc, lastMapReduceIdForSubmit); - } - } - - @Override - protected Runnable newTask(LinkedTaskContext context) { - return DaosReader.ReadTask.newInstance((DaosReader.ReadTaskContext) context); - } - - @Override - protected LinkedTaskContext createTaskContext(IODataDesc desc, Object morePara) { - return new DaosReader.ReadTaskContext(object, counter, lock, condition, desc, morePara); - } - - private ByteBuf getBySelf(IODataDesc desc, Tuple2 mapreduceId) throws IOException { - // get data by self, no need to release currentDesc - if (desc == null) { // reach end - return null; - } - boolean releaseBuf = false; - try { - object.fetch(desc); - currentDesc = desc; - ByteBuf buf = validateLastEntryAndGetBuf(desc.getEntry(entryIdx)); - lastMapReduceIdForReturn = mapreduceId; - return buf; - } catch (IOException | IllegalStateException e) { - releaseBuf = true; - throw e; - } finally { - desc.release(releaseBuf); - } - } - - private IODataDesc createNextDesc(long sizeLimit) throws IOException { - long remaining = sizeLimit; - int reduceId = -1; - long mapId; - IODataDesc desc = null; - while (remaining > 0) { - nextMapReduceId(); - if (curMapReduceId == null) { - break; - } - if (reduceId > 0 && curMapReduceId._2 != reduceId) { // make sure entries under same reduce - break; - } - reduceId = curMapReduceId._2; - mapId = curMapReduceId._1; - lastMapReduceIdForSubmit = curMapReduceId; - long readSize = partSizeMap.get(curMapReduceId)._1() - curOffset; - long offset = curOffset; - if (readSize > remaining) { - readSize = remaining; - curOffset += readSize; - } else { - curOffset = 0; - curMapReduceId = null; - } - if (desc == null) { - desc = object.createDataDescForFetch(String.valueOf(reduceId), IODataDesc.IodType.ARRAY, 1); - } - desc.addEntryForFetch(String.valueOf(mapId), (int)offset, (int)readSize); - remaining -= readSize; - } - return desc; - } - - private void nextMapReduceId() { - if (curMapReduceId != null) { - return; - } - curOffset = 0; - if (mapIdIt.hasNext()) { - curMapReduceId = mapIdIt.next(); - partsRead++; - } else { - curMapReduceId = null; - } - } - - private ByteBuf validateLastEntryAndGetBuf(IODataDesc.Entry entry) throws IOException { - ByteBuf buf = entry.getFetchedData(); - int byteLen = buf.readableBytes(); - newMap = false; - if (currentEntry != null && entry != currentEntry) { - if (entry.getKey().equals(currentEntry.getKey())) { - currentPartSize += byteLen; - } else { - checkPartitionSize(); - newMap = true; - currentPartSize = byteLen; - } - } - currentEntry = entry; - metrics.incRemoteBytesRead(byteLen); - return buf; - } - - private void checkPartitionSize() throws IOException { - if (lastMapReduceIdForReturn == null) { - return; - } - // partition size is not accurate after compress/decompress - long size = partSizeMap.get(lastMapReduceIdForReturn)._1(); - if (size < 35 * 1024 * 1024 * 1024 && currentPartSize * 1.1 < size) { - throw new IOException("expect partition size " + partSizeMap.get(lastMapReduceIdForReturn) + - ", actual size " + currentPartSize + ", mapId and reduceId: " + lastMapReduceIdForReturn); - } - metrics.incRemoteBlocksFetched(1); - } - - public boolean cleanup(boolean force) { - boolean allReleased = true; - if (!cleaned) { - allReleased &= cleanupSubmitted(force); - allReleased &= cleanupConsumed(force); - } - return allReleased; - } - - public void checkTotalPartitions() throws IOException { - if (partsRead != totalParts) { - throw new IOException("expect total partitions to be read: " + totalParts + ", actual read: " + partsRead); - } - } - - @Override - public DaosReader.ReadTaskContext getCurrentCtx() { - return (DaosReader.ReadTaskContext) currentCtx; - } - - @Override - public DaosReader.ReadTaskContext getHeadCtx() { - return (DaosReader.ReadTaskContext) headCtx; - } - - @Override - public DaosReader.ReadTaskContext getLastCtx() { - return (DaosReader.ReadTaskContext) lastCtx; - } - } - - /** - * reader configurations, please check configs prefixed with SHUFFLE_DAOS_READ in {@link package$#MODULE$}. - */ - private static final class ReaderConfig { - private long minReadSize; - private long maxBytesInFlight; - private long maxMem; - private int readBatchSize; - private int waitDataTimeMs; - private int waitTimeoutTimes; - private boolean fromOtherThread; - - private ReaderConfig(long maxBytesInFlight, long maxMem) { - SparkConf conf = SparkEnv.get().conf(); - minReadSize = (long)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_MINIMUM_SIZE()) * 1024; - if (maxBytesInFlight < minReadSize) { - this.maxBytesInFlight = minReadSize; - } else { - this.maxBytesInFlight = maxBytesInFlight; - } - this.maxMem = maxMem; - this.readBatchSize = (int)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_BATCH_SIZE()); - this.waitDataTimeMs = (int)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_WAIT_DATA_MS()); - this.waitTimeoutTimes = (int)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_WAIT_DATA_TIMEOUT_TIMES()); - this.fromOtherThread = (boolean)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_FROM_OTHER_THREAD()); - if (log.isDebugEnabled()) { - log.debug("minReadSize: " + minReadSize); - log.debug("maxBytesInFlight: " + maxBytesInFlight); - log.debug("maxMem: " + maxMem); - log.debug("readBatchSize: " + readBatchSize); - log.debug("waitDataTimeMs: " + waitDataTimeMs); - log.debug("waitTimeoutTimes: " + waitTimeoutTimes); - log.debug("fromOtherThread: " + fromOtherThread); - } - } - } } diff --git a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosWriter.java b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosWriter.java index 8b9714f2..f3922294 100644 --- a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosWriter.java +++ b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosWriter.java @@ -23,80 +23,20 @@ package org.apache.spark.shuffle.daos; -import io.daos.BufferAllocator; -import io.daos.DaosIOException; -import io.daos.obj.DaosObject; -import io.daos.obj.IODataDesc; -import io.netty.buffer.ByteBuf; -import io.netty.util.internal.ObjectPool; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkEnv; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.*; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; /** * A DAOS writer per map task which may have multiple map output partitions. - * Each partition has one corresponding {@link NativeBuffer} which caches records until - * a specific {@link #flush(int)} call being made. Then {@link NativeBuffer} creates - * {@link IODataDesc} and write to DAOS in either caller thread or other dedicated thread. + * Each partition has one corresponding buffer which caches records until + * a specific {@link #flush(int)} call being made. Data is written to DAOS + * in either caller thread or other dedicated thread. */ -public class DaosWriter extends TaskSubmitter { - - private DaosObject object; - - private String mapId; - - private WriteParam param; - - private WriteConfig config; - - private Map writerMap; - - private NativeBuffer[] partitionBufArray; - - private int totalTimeoutTimes; - - private int totalWriteTimes; - - private int totalBySelfTimes; - - private volatile boolean cleaned; - - private static Logger LOG = LoggerFactory.getLogger(DaosWriter.class); - - /** - * construct DaosWriter with object and dedicated read executors. - * - * @param param - * write parameters - * @param object - * opened DaosObject - * @param executor - * null means write in caller's thread. Submit {@link WriteTask} to it otherwise. - */ - public DaosWriter(DaosWriter.WriteParam param, DaosObject object, - BoundThreadExecutors.SingleThreadExecutor executor) { - super(executor); - this.param = param; - this.config = param.config; - this.partitionBufArray = new NativeBuffer[param.numPartitions]; - this.mapId = String.valueOf(param.mapId); - this.object = object; - } - - private NativeBuffer getNativeBuffer(int partitionId) { - NativeBuffer buffer = partitionBufArray[partitionId]; - if (buffer == null) { - buffer = new NativeBuffer(partitionId, config.bufferSize); - partitionBufArray[partitionId] = buffer; - } - return buffer; - } +public interface DaosWriter { /** * write to buffer. @@ -104,9 +44,7 @@ private NativeBuffer getNativeBuffer(int partitionId) { * @param partitionId * @param b */ - public void write(int partitionId, int b) { - getNativeBuffer(partitionId).write(b); - } + void write(int partitionId, int b); /** * write to buffer. @@ -114,9 +52,7 @@ public void write(int partitionId, int b) { * @param partitionId * @param array */ - public void write(int partitionId, byte[] array) { - getNativeBuffer(partitionId).write(array); - } + void write(int partitionId, byte[] array); /** * write to buffer. @@ -126,9 +62,7 @@ public void write(int partitionId, byte[] array) { * @param offset * @param len */ - public void write(int partitionId, byte[] array, int offset, int len) { - getNativeBuffer(partitionId).write(array, offset, len); - } + void write(int partitionId, byte[] array, int offset, int len); /** * get length of all partitions. @@ -137,32 +71,7 @@ public void write(int partitionId, byte[] array, int offset, int len) { * @param numPartitions * @return array of partition lengths */ - public long[] getPartitionLens(int numPartitions) { - if (LOG.isDebugEnabled()) { - LOG.debug("partition map size: " + partitionBufArray.length); - for (int i = 0; i < numPartitions; i++) { - NativeBuffer nb = partitionBufArray[i]; - if (nb != null) { - LOG.debug("id: " + i + ", native buffer: " + nb.partitionId + ", " + - nb.totalSize + ", " + nb.roundSize); - } - } - } - long[] lens = new long[numPartitions]; - for (int i = 0; i < numPartitions; i++) { - NativeBuffer nb = partitionBufArray[i]; - if (nb != null) { - lens[i] = nb.totalSize; - if (nb.roundSize != 0 || !nb.bufList.isEmpty()) { - throw new IllegalStateException("round size should be 0, " + nb.roundSize + ", buflist should be empty, " + - nb.bufList.size()); - } - } else { - lens[i] = 0; - } - } - return lens; - } + long[] getPartitionLens(int numPartitions); /** * Flush specific partition to DAOS. @@ -170,280 +79,17 @@ public long[] getPartitionLens(int numPartitions) { * @param partitionId * @throws IOException */ - public void flush(int partitionId) throws IOException { - NativeBuffer buffer = partitionBufArray[partitionId]; - if (buffer == null) { - return; - } - IODataDesc desc = buffer.createUpdateDesc(); - if (desc == null) { - return; - } - totalWriteTimes++; - if (config.warnSmallWrite && buffer.roundSize < config.minSize) { - LOG.warn("too small partition size {}, shuffle {}, map {}, partition {}", - buffer.roundSize, param.shuffleId, mapId, partitionId); - } - if (executor == null) { // run write by self - runBySelf(desc, buffer); - return; - } - submitToOtherThreads(desc, buffer); - } - - private void runBySelf(IODataDesc desc, NativeBuffer buffer) throws IOException { - totalBySelfTimes++; - try { - object.update(desc); - } catch (IOException e) { - throw new IOException("failed to write partition of " + desc, e); - } finally { - desc.release(); - buffer.reset(true); - } - } - - private void submitToOtherThreads(IODataDesc desc, NativeBuffer buffer) throws IOException { - // move forward to release write buffers - moveForward(); - // check if we need to wait submitted tasks to be executed - if (goodForSubmit()) { - submitAndReset(desc, buffer); - return; - } - // to wait - int timeoutTimes = 0; - try { - while (!goodForSubmit()) { - boolean timeout = waitForCondition(config.waitTimeMs); - moveForward(); - if (timeout) { - timeoutTimes++; - if (LOG.isDebugEnabled()) { - LOG.debug("wait daos write timeout times: " + timeoutTimes); - } - if (timeoutTimes >= config.timeoutTimes) { - totalTimeoutTimes += timeoutTimes; - runBySelf(desc, buffer); - return; - } - } - } - } catch (InterruptedException e) { - desc.release(); - Thread.currentThread().interrupt(); - throw new IOException("interrupted when wait daos write", e); - } - // submit write task after some wait - totalTimeoutTimes += timeoutTimes; - submitAndReset(desc, buffer); - } - - private boolean goodForSubmit() { - return totalInMemSize < config.totalInMemSize && totalSubmitted < config.totalSubmittedLimit; - } - - private void submitAndReset(IODataDesc desc, NativeBuffer buffer) { - try { - submit(desc, buffer.bufList); - } finally { - buffer.reset(false); - } - } - - private void cleanup(boolean force) { - if (cleaned) { - return; - } - boolean allReleased = true; - allReleased &= cleanupSubmitted(force); - allReleased &= cleanupConsumed(force); - if (allReleased) { - cleaned = true; - } - } + void flush(int partitionId) throws IOException; /** - * wait write task to be completed and clean up resources. + * close writer. */ - public void close() { - try { - close(true); - } catch (Exception e) { - throw new IllegalStateException("failed to complete all write tasks and cleanup", e); - } - } - - private void close(boolean force) throws Exception { - if (partitionBufArray != null) { - waitCompletion(force); - partitionBufArray = null; - object = null; - if (LOG.isDebugEnabled()) { - LOG.debug("total writes: " + totalWriteTimes + ", total timeout times: " + totalTimeoutTimes + - ", total write-by-self times: " + totalBySelfTimes + ", total timeout times/total writes: " + - ((float) totalTimeoutTimes) / totalWriteTimes); - } - } - cleanup(force); - if (writerMap != null && (force || cleaned)) { - writerMap.remove(this); - writerMap = null; - } - } - - private void waitCompletion(boolean force) throws Exception { - if (!force) { - return; - } - try { - while (totalSubmitted > 0) { - waitForCondition(config.waitTimeMs); - moveForward(); - } - } catch (Exception e) { - LOG.error("failed to wait completion of daos writing", e); - throw e; - } - } - - public void setWriterMap(Map writerMap) { - writerMap.put(this, 0); - this.writerMap = writerMap; - } - - @Override - protected Runnable newTask(LinkedTaskContext context) { - return WriteTask.newInstance((WriteTaskContext) context); - } - - @Override - protected LinkedTaskContext createTaskContext(IODataDesc desc, Object morePara) { - return new WriteTaskContext(object, counter, lock, condition, desc, morePara); - } - - @Override - protected boolean validateReturned(LinkedTaskContext context) throws IOException { - if (!context.desc.isSucceeded()) { - throw new DaosIOException("write is not succeeded: " + context.desc); - } - return false; - } - - @Override - protected boolean consumed(LinkedTaskContext context) { - // release write buffers - List bufList = (List) context.morePara; - bufList.forEach(b -> b.release()); - bufList.clear(); - return true; - } - - /** - * Write data to one or multiple netty direct buffers which will be written to DAOS without copy - */ - private class NativeBuffer implements Comparable { - private int partitionId; - private String partitionIdKey; - private int bufferSize; - private int idx = -1; - private List bufList = new ArrayList<>(); - private long totalSize; - private long roundSize; - - NativeBuffer(int partitionId, int bufferSize) { - this.partitionId = partitionId; - this.partitionIdKey = String.valueOf(partitionId); - this.bufferSize = bufferSize; - } - - private ByteBuf addNewByteBuf(int len) { - ByteBuf buf; - try { - buf = BufferAllocator.objBufWithNativeOrder(Math.max(bufferSize, len)); - } catch (OutOfMemoryError e) { - LOG.error("too big buffer size: " + Math.max(bufferSize, len)); - throw e; - } - bufList.add(buf); - idx++; - return buf; - } - - private ByteBuf getBuffer(int len) { - if (idx < 0) { - return addNewByteBuf(len); - } - return bufList.get(idx); - } - - public void write(int b) { - ByteBuf buf = getBuffer(1); - if (buf.writableBytes() < 1) { - buf = addNewByteBuf(1); - } - buf.writeByte(b); - roundSize += 1; - } - - public void write(byte[] b) { - write(b, 0, b.length); - } - - public void write(byte[] b, int offset, int len) { - if (len <= 0) { - return; - } - ByteBuf buf = getBuffer(len); - int avail = buf.writableBytes(); - int gap = len - avail; - if (gap <= 0) { - buf.writeBytes(b, offset, len); - } else { - buf.writeBytes(b, offset, avail); - buf = addNewByteBuf(gap); - buf.writeBytes(b, avail, gap); - } - roundSize += len; - } - - public IODataDesc createUpdateDesc() throws IOException { - if (roundSize == 0 || bufList.isEmpty()) { - return null; - } - long bufSize = 0; - IODataDesc desc = object.createDataDescForUpdate(partitionIdKey, IODataDesc.IodType.ARRAY, 1); - for (ByteBuf buf : bufList) { - desc.addEntryForUpdate(mapId, (int) totalSize, buf); - bufSize += buf.readableBytes(); - } - if (roundSize != bufSize) { - throw new IOException("expect update size: " + roundSize + ", actual: " + bufSize); - } - return desc; - } - - public void reset(boolean release) { - if (release) { - bufList.forEach(b -> b.release()); - } - // release==false, buffers will be released when tasks are executed and consumed - bufList.clear(); - idx = -1; - totalSize += roundSize; - roundSize = 0; - } - - @Override - public int compareTo(NativeBuffer nativeBuffer) { - return partitionId - nativeBuffer.partitionId; - } - } + void close(); /** * Write configurations. Please check configs prefixed with SHUFFLE_DAOS_WRITE in {@link package$#MODULE$}. */ - public static class WriteConfig { + class WriterConfig { private int bufferSize; private int minSize; private boolean warnSmallWrite; @@ -454,50 +100,26 @@ public static class WriteConfig { private int threads; private boolean fromOtherThreads; - public WriteConfig bufferSize(int bufferSize) { - this.bufferSize = bufferSize; - return this; - } - - public WriteConfig minSize(int minSize) { - this.minSize = minSize; - return this; - } - - public WriteConfig warnSmallWrite(boolean warnSmallWrite) { - this.warnSmallWrite = warnSmallWrite; - return this; - } - - public WriteConfig waitTimeMs(long waitTimeMs) { - this.waitTimeMs = waitTimeMs; - return this; - } - - public WriteConfig timeoutTimes(int timeoutTimes) { - this.timeoutTimes = timeoutTimes; - return this; - } - - public WriteConfig totalInMemSize(long totalInMemSize) { - this.totalInMemSize = totalInMemSize; - return this; - } + private static final Logger logger = LoggerFactory.getLogger(WriterConfig.class); - public WriteConfig totalSubmittedLimit(int totalSubmittedLimit) { - this.totalSubmittedLimit = totalSubmittedLimit; - return this; - } - - public WriteConfig threads(int threads) { - this.threads = threads; - return this; + WriterConfig() { + SparkConf conf = SparkEnv.get().conf(); + warnSmallWrite = (boolean) conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_WARN_SMALL_SIZE()); + bufferSize = (int) ((long) conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_SINGLE_BUFFER_SIZE()) + * 1024 * 1024); + minSize = (int) ((long)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_MINIMUM_SIZE()) * 1024); + timeoutTimes = (int)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_WAIT_DATA_TIMEOUT_TIMES()); + waitTimeMs = (int)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_WAIT_MS()); + totalInMemSize = (long)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_MAX_BYTES_IN_FLIGHT()) * 1024; + totalSubmittedLimit = (int)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_SUBMITTED_LIMIT()); + threads = (int)conf.get(package$.MODULE$.SHUFFLE_DAOS_WRITE_THREADS()); + fromOtherThreads = (boolean)conf + .get(package$.MODULE$.SHUFFLE_DAOS_WRITE_IN_OTHER_THREAD()); + if (logger.isDebugEnabled()) { + logger.debug(toString()); + } } - public WriteConfig fromOtherThreads(boolean fromOtherThreads) { - this.fromOtherThreads = fromOtherThreads; - return this; - } public int getBufferSize() { return bufferSize; @@ -550,141 +172,4 @@ public String toString() { '}'; } } - - public static class WriteParam { - private int numPartitions; - private int shuffleId; - private long mapId; - private WriteConfig config; - - public WriteParam numPartitions(int numPartitions) { - this.numPartitions = numPartitions; - return this; - } - - public WriteParam shuffleId(int shuffleId) { - this.shuffleId = shuffleId; - return this; - } - - public WriteParam mapId(long mapId) { - this.mapId = mapId; - return this; - } - - public WriteParam config(WriteConfig config) { - this.config = config; - return this; - } - } - - /** - * Task to write data to DAOS. Task itself is cached to reduce GC time. - * To reuse task for different writes, prepare and reset {@link WriteTaskContext} by calling - * {@link #newInstance(WriteTaskContext)} - */ - static final class WriteTask implements Runnable { - private final ObjectPool.Handle handle; - private WriteTaskContext context; - - private static final ObjectPool objectPool = ObjectPool.newPool(handle -> new WriteTask(handle)); - - private static final Logger log = LoggerFactory.getLogger(WriteTask.class); - - static WriteTask newInstance(WriteTaskContext context) { - WriteTask task = objectPool.get(); - task.context = context; - return task; - } - - private WriteTask(ObjectPool.Handle handle) { - this.handle = handle; - } - - @Override - public void run() { - boolean cancelled = context.cancelled; - try { - if (!cancelled) { - context.object.update(context.desc); - } - } catch (Exception e) { - log.error("failed to write for " + context.desc, e); - } finally { - context.desc.release(); - context.signal(); - context = null; - handle.recycle(this); - } - } - } - - /** - * Context for write task. It holds all other object to read and sync between caller thread and write thread. - * It should be cached in caller thread for reusing. - */ - static final class WriteTaskContext extends LinkedTaskContext { - - /** - * constructor with all parameters. Some of them can be reused later. - * - * @param object - * DAOS object to fetch data from DAOS - * @param counter - * counter to indicate how many write is on-going - * @param writeLock - * lock to work with notFull condition to signal caller thread to submit more write task - * @param notFull - * condition to signal caller thread - * @param desc - * desc object to describe where to write data - * @param bufList - * list of buffers to write to DAOS - */ - WriteTaskContext(DaosObject object, AtomicInteger counter, Lock writeLock, Condition notFull, - IODataDesc desc, Object bufList) { - super(object, counter, writeLock, notFull); - this.desc = desc; - List myBufList = new ArrayList<>(); - myBufList.addAll((List) bufList); - this.morePara = myBufList; - } - - @Override - public WriteTaskContext getNext() { - return (WriteTaskContext) next; - } - - @Override - public void reuse(IODataDesc desc, Object morePara) { - List myBufList = (List) this.morePara; - if (!myBufList.isEmpty()) { - throw new IllegalStateException("bufList in reusing write task context should be empty"); - } - myBufList.addAll((List) morePara); - super.reuse(desc, myBufList); - } - } - - /** - * Thread factory for write - */ - protected static class WriteThreadFactory implements ThreadFactory { - private AtomicInteger id = new AtomicInteger(0); - - @Override - public Thread newThread(Runnable runnable) { - Thread t; - String name = "daos_write_" + id.getAndIncrement(); - if (runnable == null) { - t = new Thread(name); - } else { - t = new Thread(runnable, name); - } - t.setDaemon(true); - t.setUncaughtExceptionHandler((thread, throwable) -> - LOG.error("exception occurred in thread " + name, throwable)); - return t; - } - } } diff --git a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosWriterSync.java b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosWriterSync.java new file mode 100644 index 00000000..b1165ce7 --- /dev/null +++ b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/DaosWriterSync.java @@ -0,0 +1,559 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos; + +import io.daos.BufferAllocator; +import io.daos.DaosIOException; +import io.daos.obj.DaosObject; +import io.daos.obj.IODataDesc; +import io.netty.buffer.ByteBuf; +import io.netty.util.internal.ObjectPool; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; + +/** + * A implementation of {@link DaosWriter} bases on synchronous DAOS Object API. + * + * For each partition, there is one corresponding {@link NativeBuffer} creates + * {@link IODataDesc} and write to DAOS in either caller thread or other dedicated thread. + */ +public class DaosWriterSync extends TaskSubmitter implements DaosWriter { + + private DaosObject object; + + private String mapId; + + private WriteParam param; + + private WriterConfig config; + + private Map writerMap; + + private NativeBuffer[] partitionBufArray; + + private int totalTimeoutTimes; + + private int totalWriteTimes; + + private int totalBySelfTimes; + + private volatile boolean cleaned; + + private static Logger LOG = LoggerFactory.getLogger(DaosWriterSync.class); + + /** + * construct DaosWriter with object and dedicated read executors. + * + * @param param + * write parameters + * @param object + * opened DaosObject + * @param executor + * null means write in caller's thread. Submit {@link WriteTask} to it otherwise. + */ + public DaosWriterSync(DaosObject object, DaosWriterSync.WriteParam param, + BoundThreadExecutors.SingleThreadExecutor executor) { + super(executor); + this.param = param; + this.config = param.config; + this.partitionBufArray = new NativeBuffer[param.numPartitions]; + this.mapId = String.valueOf(param.mapId); + this.object = object; + } + + private NativeBuffer getNativeBuffer(int partitionId) { + NativeBuffer buffer = partitionBufArray[partitionId]; + if (buffer == null) { + buffer = new NativeBuffer(partitionId, config.getBufferSize()); + partitionBufArray[partitionId] = buffer; + } + return buffer; + } + + @Override + public void write(int partitionId, int b) { + getNativeBuffer(partitionId).write(b); + } + + @Override + public void write(int partitionId, byte[] array) { + getNativeBuffer(partitionId).write(array); + } + + @Override + public void write(int partitionId, byte[] array, int offset, int len) { + getNativeBuffer(partitionId).write(array, offset, len); + } + + @Override + public long[] getPartitionLens(int numPartitions) { + if (LOG.isDebugEnabled()) { + LOG.debug("partition map size: " + partitionBufArray.length); + for (int i = 0; i < numPartitions; i++) { + NativeBuffer nb = partitionBufArray[i]; + if (nb != null) { + LOG.debug("id: " + i + ", native buffer: " + nb.partitionId + ", " + + nb.totalSize + ", " + nb.roundSize); + } + } + } + long[] lens = new long[numPartitions]; + for (int i = 0; i < numPartitions; i++) { + NativeBuffer nb = partitionBufArray[i]; + if (nb != null) { + lens[i] = nb.totalSize; + if (nb.roundSize != 0 || !nb.bufList.isEmpty()) { + throw new IllegalStateException("round size should be 0, " + nb.roundSize + ", buflist should be empty, " + + nb.bufList.size()); + } + } else { + lens[i] = 0; + } + } + return lens; + } + + @Override + public void flush(int partitionId) throws IOException { + NativeBuffer buffer = partitionBufArray[partitionId]; + if (buffer == null) { + return; + } + IODataDesc desc = buffer.createUpdateDesc(); + if (desc == null) { + return; + } + totalWriteTimes++; + if (config.isWarnSmallWrite() && buffer.roundSize < config.getMinSize()) { + LOG.warn("too small partition size {}, shuffle {}, map {}, partition {}", + buffer.roundSize, param.shuffleId, mapId, partitionId); + } + if (executor == null) { // run write by self + runBySelf(desc, buffer); + return; + } + submitToOtherThreads(desc, buffer); + } + + private void runBySelf(IODataDesc desc, NativeBuffer buffer) throws IOException { + totalBySelfTimes++; + try { + object.update(desc); + } catch (IOException e) { + throw new IOException("failed to write partition of " + desc, e); + } finally { + desc.release(); + buffer.reset(true); + } + } + + private void submitToOtherThreads(IODataDesc desc, NativeBuffer buffer) throws IOException { + // move forward to release write buffers + moveForward(); + // check if we need to wait submitted tasks to be executed + if (goodForSubmit()) { + submitAndReset(desc, buffer); + return; + } + // to wait + int timeoutTimes = 0; + try { + while (!goodForSubmit()) { + boolean timeout = waitForCondition(config.getWaitTimeMs()); + moveForward(); + if (timeout) { + timeoutTimes++; + if (LOG.isDebugEnabled()) { + LOG.debug("wait daos write timeout times: " + timeoutTimes); + } + if (timeoutTimes >= config.getTimeoutTimes()) { + totalTimeoutTimes += timeoutTimes; + runBySelf(desc, buffer); + return; + } + } + } + } catch (InterruptedException e) { + desc.release(); + Thread.currentThread().interrupt(); + throw new IOException("interrupted when wait daos write", e); + } + // submit write task after some wait + totalTimeoutTimes += timeoutTimes; + submitAndReset(desc, buffer); + } + + private boolean goodForSubmit() { + return totalInMemSize < config.getTotalInMemSize() && totalSubmitted < config.getTotalSubmittedLimit(); + } + + private void submitAndReset(IODataDesc desc, NativeBuffer buffer) { + try { + submit(desc, buffer.bufList); + } finally { + buffer.reset(false); + } + } + + private void cleanup(boolean force) { + if (cleaned) { + return; + } + boolean allReleased = true; + allReleased &= cleanupSubmitted(force); + allReleased &= cleanupConsumed(force); + if (allReleased) { + cleaned = true; + } + } + + /** + * wait write task to be completed and clean up resources. + */ + @Override + public void close() { + try { + close(true); + } catch (Exception e) { + throw new IllegalStateException("failed to complete all write tasks and cleanup", e); + } + } + + private void close(boolean force) throws Exception { + if (partitionBufArray != null) { + waitCompletion(force); + partitionBufArray = null; + object = null; + if (LOG.isDebugEnabled()) { + LOG.debug("total writes: " + totalWriteTimes + ", total timeout times: " + totalTimeoutTimes + + ", total write-by-self times: " + totalBySelfTimes + ", total timeout times/total writes: " + + ((float) totalTimeoutTimes) / totalWriteTimes); + } + } + cleanup(force); + if (writerMap != null && (force || cleaned)) { + writerMap.remove(this); + writerMap = null; + } + } + + private void waitCompletion(boolean force) throws Exception { + if (!force) { + return; + } + try { + while (totalSubmitted > 0) { + waitForCondition(config.getWaitTimeMs()); + moveForward(); + } + } catch (Exception e) { + LOG.error("failed to wait completion of daos writing", e); + throw e; + } + } + + public void setWriterMap(Map writerMap) { + writerMap.put(this, 0); + this.writerMap = writerMap; + } + + @Override + protected Runnable newTask(LinkedTaskContext context) { + return WriteTask.newInstance((WriteTaskContext) context); + } + + @Override + protected LinkedTaskContext createTaskContext(IODataDesc desc, Object morePara) { + return new WriteTaskContext(object, counter, lock, condition, desc, morePara); + } + + @Override + protected boolean validateReturned(LinkedTaskContext context) throws IOException { + if (!context.desc.isSucceeded()) { + throw new DaosIOException("write is not succeeded: " + context.desc); + } + return false; + } + + @Override + protected boolean consumed(LinkedTaskContext context) { + // release write buffers + @SuppressWarnings("unchecked") + List bufList = (List) context.morePara; + bufList.forEach(b -> b.release()); + bufList.clear(); + return true; + } + + /** + * Write data to one or multiple netty direct buffers which will be written to DAOS without copy + */ + private class NativeBuffer implements Comparable { + private int partitionId; + private String partitionIdKey; + private int bufferSize; + private int idx = -1; + private List bufList = new ArrayList<>(); + private long totalSize; + private long roundSize; + + NativeBuffer(int partitionId, int bufferSize) { + this.partitionId = partitionId; + this.partitionIdKey = String.valueOf(partitionId); + this.bufferSize = bufferSize; + } + + private ByteBuf addNewByteBuf(int len) { + ByteBuf buf; + try { + buf = BufferAllocator.objBufWithNativeOrder(Math.max(bufferSize, len)); + } catch (OutOfMemoryError e) { + LOG.error("too big buffer size: " + Math.max(bufferSize, len)); + throw e; + } + bufList.add(buf); + idx++; + return buf; + } + + private ByteBuf getBuffer(int len) { + if (idx < 0) { + return addNewByteBuf(len); + } + return bufList.get(idx); + } + + public void write(int b) { + ByteBuf buf = getBuffer(1); + if (buf.writableBytes() < 1) { + buf = addNewByteBuf(1); + } + buf.writeByte(b); + roundSize += 1; + } + + public void write(byte[] b) { + write(b, 0, b.length); + } + + public void write(byte[] b, int offset, int len) { + if (len <= 0) { + return; + } + ByteBuf buf = getBuffer(len); + int avail = buf.writableBytes(); + int gap = len - avail; + if (gap <= 0) { + buf.writeBytes(b, offset, len); + } else { + buf.writeBytes(b, offset, avail); + buf = addNewByteBuf(gap); + buf.writeBytes(b, avail, gap); + } + roundSize += len; + } + + public IODataDesc createUpdateDesc() throws IOException { + if (roundSize == 0 || bufList.isEmpty()) { + return null; + } + long bufSize = 0; + IODataDesc desc = object.createDataDescForUpdate(partitionIdKey, IODataDesc.IodType.ARRAY, 1); + for (ByteBuf buf : bufList) { + desc.addEntryForUpdate(mapId, (int) totalSize, buf); + bufSize += buf.readableBytes(); + } + if (roundSize != bufSize) { + throw new IOException("expect update size: " + roundSize + ", actual: " + bufSize); + } + return desc; + } + + public void reset(boolean release) { + if (release) { + bufList.forEach(b -> b.release()); + } + // release==false, buffers will be released when tasks are executed and consumed + bufList.clear(); + idx = -1; + totalSize += roundSize; + roundSize = 0; + } + + @Override + public int compareTo(NativeBuffer nativeBuffer) { + return partitionId - nativeBuffer.partitionId; + } + } + + public static class WriteParam { + private int numPartitions; + private int shuffleId; + private long mapId; + private DaosWriter.WriterConfig config; + + public WriteParam numPartitions(int numPartitions) { + this.numPartitions = numPartitions; + return this; + } + + public WriteParam shuffleId(int shuffleId) { + this.shuffleId = shuffleId; + return this; + } + + public WriteParam mapId(long mapId) { + this.mapId = mapId; + return this; + } + + public WriteParam config(DaosWriter.WriterConfig config) { + this.config = config; + return this; + } + } + + /** + * Task to write data to DAOS. Task itself is cached to reduce GC time. + * To reuse task for different writes, prepare and reset {@link WriteTaskContext} by calling + * {@link #newInstance(WriteTaskContext)} + */ + static final class WriteTask implements Runnable { + private final ObjectPool.Handle handle; + private WriteTaskContext context; + + private static final ObjectPool objectPool = ObjectPool.newPool(handle -> new WriteTask(handle)); + + private static final Logger log = LoggerFactory.getLogger(WriteTask.class); + + static WriteTask newInstance(WriteTaskContext context) { + WriteTask task = objectPool.get(); + task.context = context; + return task; + } + + private WriteTask(ObjectPool.Handle handle) { + this.handle = handle; + } + + @Override + public void run() { + boolean cancelled = context.cancelled; + try { + if (!cancelled) { + context.object.update(context.desc); + } + } catch (Exception e) { + log.error("failed to write for " + context.desc, e); + } finally { + context.desc.release(); + context.signal(); + context = null; + handle.recycle(this); + } + } + } + + /** + * Context for write task. It holds all other object to read and sync between caller thread and write thread. + * It should be cached in caller thread for reusing. + */ + static final class WriteTaskContext extends LinkedTaskContext { + + /** + * constructor with all parameters. Some of them can be reused later. + * + * @param object + * DAOS object to fetch data from DAOS + * @param counter + * counter to indicate how many write is on-going + * @param writeLock + * lock to work with notFull condition to signal caller thread to submit more write task + * @param notFull + * condition to signal caller thread + * @param desc + * desc object to describe where to write data + * @param bufList + * list of buffers to write to DAOS + */ + WriteTaskContext(DaosObject object, AtomicInteger counter, Lock writeLock, Condition notFull, + IODataDesc desc, Object bufList) { + super(object, counter, writeLock, notFull); + this.desc = desc; + @SuppressWarnings("unchecked") + List myBufList = new ArrayList<>(); + myBufList.addAll((List) bufList); + this.morePara = myBufList; + } + + @Override + public WriteTaskContext getNext() { + @SuppressWarnings("unchecked") + WriteTaskContext ctx = (WriteTaskContext) next; + return ctx; + } + + @Override + public void reuse(IODataDesc desc, Object morePara) { + @SuppressWarnings("unchecked") + List myBufList = (List) this.morePara; + if (!myBufList.isEmpty()) { + throw new IllegalStateException("bufList in reusing write task context should be empty"); + } + myBufList.addAll((List) morePara); + super.reuse(desc, myBufList); + } + } + + /** + * Thread factory for write + */ + protected static class WriteThreadFactory implements ThreadFactory { + private AtomicInteger id = new AtomicInteger(0); + + @Override + public Thread newThread(Runnable runnable) { + Thread t; + String name = "daos_write_" + id.getAndIncrement(); + if (runnable == null) { + t = new Thread(name); + } else { + t = new Thread(runnable, name); + } + t.setDaemon(true); + t.setUncaughtExceptionHandler((thread, throwable) -> + LOG.error("exception occurred in thread " + name, throwable)); + return t; + } + } +} diff --git a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/IOManager.java b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/IOManager.java new file mode 100644 index 00000000..aebea83f --- /dev/null +++ b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/IOManager.java @@ -0,0 +1,88 @@ +/* + * (C) Copyright 2018-2020 Intel Corporation. + * + * Licensed 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. + * + * GOVERNMENT LICENSE RIGHTS-OPEN SOURCE SOFTWARE + * The Government's rights to use, modify, reproduce, release, perform, display, + * or disclose this software are subject to the terms of the Apache License as + * provided in Contract No. B609815. + * Any reproduction of computer software, computer software documentation, or + * portions thereof marked with this legend must also reproduce the markings. + */ + +package org.apache.spark.shuffle.daos; + +import io.daos.obj.DaosObjClient; +import io.daos.obj.DaosObject; +import io.daos.obj.DaosObjectException; +import io.daos.obj.DaosObjectId; +import org.apache.spark.SparkConf; + +import java.io.IOException; +import java.util.Map; + +public abstract class IOManager { + + protected Map objectMap; + + protected SparkConf conf; + + protected DaosObjClient objClient; + + protected IOManager(SparkConf conf, Map objectMap) { + this.conf = conf; + this.objectMap = objectMap; + } + + private String getKey(long appId, int shuffleId) { + return appId + "" + shuffleId; + } + + protected static long parseAppId(String appId) { + return Long.valueOf(appId.replaceAll("\\D", "")); + } + + protected DaosObject getObject(long appId, int shuffleId) throws DaosObjectException { + String key = getKey(appId, shuffleId); + DaosObject object = objectMap.get(key); + if (object == null) { + DaosObjectId id = new DaosObjectId(appId, shuffleId); + id.encode(); + object = objClient.getObject(id); + objectMap.putIfAbsent(key, object); + DaosObject activeObject = objectMap.get(key); + if (activeObject != object) { // release just created DaosObject + object.close(); + object = activeObject; + } + } + // open just once in multiple threads + if (!object.isOpen()) { + synchronized (object) { + object.open(); + } + } + return object; + } + + public void setObjClient(DaosObjClient objClient) { + this.objClient = objClient; + } + + abstract DaosWriter getDaosWriter(int numPartitions, int shuffleId, long mapId) throws IOException; + + abstract DaosReader getDaosReader(int shuffleId) throws IOException; + + abstract void close() throws IOException; +} diff --git a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/IOManagerAsync.java b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/IOManagerAsync.java new file mode 100644 index 00000000..122493c3 --- /dev/null +++ b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/IOManagerAsync.java @@ -0,0 +1,29 @@ +package org.apache.spark.shuffle.daos; + +import io.daos.obj.DaosObject; +import org.apache.spark.SparkConf; + +import java.io.IOException; +import java.util.Map; + +public class IOManagerAsync extends IOManager { + + public IOManagerAsync(SparkConf conf, Map objectMap) { + super(conf, objectMap); + } + + @Override + DaosWriter getDaosWriter(int numPartitions, int shuffleId, long mapId) throws IOException { + return null; + } + + @Override + DaosReader getDaosReader(int shuffleId) throws IOException { + return null; + } + + @Override + void close() throws IOException { + + } +} diff --git a/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/IOManagerSync.java b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/IOManagerSync.java new file mode 100644 index 00000000..d49f79bf --- /dev/null +++ b/shuffle-daos/src/main/java/org/apache/spark/shuffle/daos/IOManagerSync.java @@ -0,0 +1,113 @@ +package org.apache.spark.shuffle.daos; + +import io.daos.obj.DaosObject; +import org.apache.spark.SparkConf; +import org.apache.spark.launcher.SparkLauncher; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class IOManagerSync extends IOManager { + + private BoundThreadExecutors readerExes; + + private BoundThreadExecutors writerExes; + + private DaosReader.ReaderConfig readerConfig; + + private DaosWriter.WriterConfig writerConfig; + + private Map readerMap = new ConcurrentHashMap<>(); + + private Map writerMap = new ConcurrentHashMap<>(); + + private Logger logger = LoggerFactory.getLogger(IOManagerSync.class); + + public IOManagerSync(SparkConf conf, Map objectMap) { + super(conf, objectMap); + readerConfig = new DaosReader.ReaderConfig(); + writerConfig = new DaosWriter.WriterConfig(); + readerExes = createReaderExes(); + writerExes = createWriterExes(); + } + + private BoundThreadExecutors createWriterExes() { + if (writerConfig.isFromOtherThreads()) { + BoundThreadExecutors executors; + int threads = writerConfig.getThreads(); + if (threads == -1) { + threads = conf.getInt(SparkLauncher.EXECUTOR_CORES, 1); + } + executors = new BoundThreadExecutors("write_executors", threads, + new DaosWriterSync.WriteThreadFactory()); + logger.info("created BoundThreadExecutors with " + threads + " threads for write"); + return executors; + } + return null; + } + + private BoundThreadExecutors createReaderExes() { + if (readerConfig.isFromOtherThread()) { + BoundThreadExecutors executors; + int threads = (int)conf.get(package$.MODULE$.SHUFFLE_DAOS_READ_THREADS()); + if (threads == -1) { + threads = conf.getInt(SparkLauncher.EXECUTOR_CORES, 1); + } + executors = new BoundThreadExecutors("read_executors", threads, + new DaosReaderSync.ReadThreadFactory()); + logger.info("created BoundThreadExecutors with " + threads + " threads for read"); + return executors; + } + return null; + } + + @Override + public DaosWriterSync getDaosWriter(int numPartitions, int shuffleId, long mapId) throws IOException { + long appId = parseAppId(conf.getAppId()); + if (logger.isDebugEnabled()) { + logger.debug("getting daoswriter for app id: " + appId + ", shuffle id: " + shuffleId + ", map id: " + mapId + + ", numPartitions: " + numPartitions); + } + DaosWriterSync.WriteParam param = new DaosWriterSync.WriteParam(); + param.numPartitions(numPartitions) + .shuffleId(shuffleId) + .mapId(mapId) + .config(writerConfig); + DaosWriterSync writer = new DaosWriterSync(getObject(appId, shuffleId), param, + writerExes == null ? null : writerExes.nextExecutor()); + writer.setWriterMap(writerMap); + return writer; + } + + @Override + DaosReader getDaosReader(int shuffleId) throws IOException { + long appId = parseAppId(conf.getAppId()); + if (logger.isDebugEnabled()) { + logger.debug("getting daosreader for app id: " + appId + ", shuffle id: " + shuffleId); + } + DaosReaderSync reader = new DaosReaderSync(getObject(appId, shuffleId), readerConfig, + readerExes == null ? null : readerExes.nextExecutor()); + reader.setReaderMap(readerMap); + return reader; + } + + @Override + void close() throws IOException { + if (readerExes != null) { + readerExes.stop(); + readerExes = null; + } + readerMap.keySet().forEach(r -> r.close(true)); + readerMap.clear(); + if (writerExes != null) { + writerExes.stop(); + writerExes = null; + } + writerMap.keySet().forEach(r -> r.close()); + writerMap.clear(); + objClient.forceClose(); + } +} diff --git a/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/MapPartitionsWriter.scala b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/MapPartitionsWriter.scala index d8ebb83e..fc7dd323 100644 --- a/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/MapPartitionsWriter.scala +++ b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/MapPartitionsWriter.scala @@ -29,7 +29,6 @@ import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.memory.{MemoryConsumer, TaskMemoryManager} import org.apache.spark.serializer.Serializer -import org.apache.spark.util.collection.SizeTrackingAppendOnlyMap class MapPartitionsWriter[K, V, C]( shuffleId: Int, @@ -88,7 +87,6 @@ class MapPartitionsWriter[K, V, C]( def insertAll(records: Iterator[Product2[K, V]]): Unit = { // TODO: stop combining if we find that the reduction factor isn't high - val start = System.nanoTime(); val shouldCombine = aggregator.isDefined if (shouldCombine) { // Combine values in-memory first using our AppendOnlyMap diff --git a/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/package.scala b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/package.scala index f38f68a9..91f8b181 100644 --- a/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/package.scala +++ b/shuffle-daos/src/main/scala/org/apache/spark/shuffle/daos/package.scala @@ -42,12 +42,6 @@ package object daos { .stringConf .createWithDefault(null) - val SHUFFLE_DAOS_POOL_RANKS = - ConfigBuilder("spark.shuffle.daos.ranks") - .version("3.0.0") - .stringConf - .createWithDefault("0") - val SHUFFLE_DAOS_REMOVE_SHUFFLE_DATA = ConfigBuilder("spark.shuffle.remove.shuffle.data") .doc("remove shuffle data from DAOS after shuffle completed. Default is true") @@ -146,6 +140,15 @@ package object daos { s"The DAOS write max bytes in flight must be positive") .createWithDefaultString("20480k") + val SHUFFLE_DAOS_IO_ASYNC = + ConfigBuilder("spark.shuffle.daos.io.async") + .doc("perform shuffle IO asynchronously. Default is true") + .version("3.0.0") + .booleanConf + .createWithDefault(true) + + /* =====configs below for DAOS synchronous API===== */ + val SHUFFLE_DAOS_READ_THREADS = ConfigBuilder("spark.shuffle.daos.read.threads") .doc("number of threads for each executor to read shuffle data concurrently. -1 means use number of executor " + diff --git a/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleIOTest.java b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleIOTest.java index 11bbdc6e..9cb77d6e 100644 --- a/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleIOTest.java +++ b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleIOTest.java @@ -26,7 +26,9 @@ import io.daos.obj.DaosObjClient; import io.daos.obj.DaosObject; import io.daos.obj.DaosObjectId; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -46,7 +48,7 @@ @RunWith(PowerMockRunner.class) @PowerMockIgnore("javax.management.*") -@PrepareForTest(DaosShuffleIO.class) +@PrepareForTest(IOManager.class) @SuppressStaticInitializationFor("io.daos.obj.DaosObjClient") public class DaosShuffleIOTest { @@ -54,11 +56,16 @@ public class DaosShuffleIOTest { public void testSingleObjectInstanceOpen() throws Exception { SparkConf testConf = new SparkConf(false); testConf.set(package$.MODULE$.SHUFFLE_DAOS_READ_FROM_OTHER_THREAD(), false); + testConf.set(package$.MODULE$.SHUFFLE_DAOS_IO_ASYNC(), false); long appId = 1234567; int shuffleId = 1; testConf.set("spark.app.id", String.valueOf(appId)); - Field clientField = DaosShuffleIO.class.getDeclaredField("objClient"); + Field clientField = IOManager.class.getDeclaredField("objClient"); clientField.setAccessible(true); + + UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser("test")); + SparkContext sc = new SparkContext("local", "test", testConf); + DaosShuffleIO io = new DaosShuffleIO(testConf); DaosObjectId id = PowerMockito.mock(DaosObjectId.class); @@ -77,7 +84,7 @@ public void testSingleObjectInstanceOpen() throws Exception { open.compareAndSet(false, true); return invocationOnMock; }).when(daosObject).open(); - clientField.set(io, client); + clientField.set(io.getIoManager(), client); int numThreads = 50; ExecutorService es = Executors.newFixedThreadPool(numThreads); @@ -102,5 +109,6 @@ public void testSingleObjectInstanceOpen() throws Exception { es.awaitTermination(5, TimeUnit.SECONDS); Assert.assertEquals(50, count.intValue()); + sc.stop(); } } diff --git a/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleInputStreamTest.java b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleInputStreamTest.java index 8b47c6d8..6c41d92b 100644 --- a/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleInputStreamTest.java +++ b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleInputStreamTest.java @@ -175,9 +175,9 @@ public void readFromOtherThreadCancelMultipleTimes(Map ma IODataDesc.Entry entry = desc.getEntry(0); String mapId = entry.getKey(); if (maps.containsKey(mapId)) { -// Thread thread = maps.get(mapId); + // Thread thread = maps.get(mapId); if (callerThread != Thread.currentThread()) { -// wait.incrementAndGet(); + // wait.incrementAndGet(); // sleep to cause read timeout System.out.println("sleeping at " + mapId); Thread.sleep(waitDataTimeMs + addWaitTimeMs); @@ -317,8 +317,8 @@ private void read(int maps, Answer answer, Mockito.doAnswer(answer).when(daosObject).fetch(any(IODataDesc.class)); BoundThreadExecutors executors = new BoundThreadExecutors("read_executors", 1, - new DaosReader.ReadThreadFactory()); - DaosReader daosReader = new DaosReader(daosObject, executors); + new DaosReaderSync.ReadThreadFactory()); + DaosReaderSync daosReader = new DaosReaderSync(daosObject, new DaosReader.ReaderConfig(), executors.nextExecutor()); LinkedHashMap, Tuple3> partSizeMap = new LinkedHashMap<>(); int shuffleId = 10; int reduceId = 1; @@ -355,7 +355,7 @@ private void read(int maps, Answer answer, System.out.println("total fetch wait time: " + taskContext.taskMetrics().shuffleReadMetrics()._fetchWaitTime().sum()); } finally { - daosReader.close(); + daosReader.close(true); is.close(true); context.stop(); if (executors != null) { diff --git a/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosWriterTest.java b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosWriterTest.java index 60c45269..3f5bc2fb 100644 --- a/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosWriterTest.java +++ b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosWriterTest.java @@ -27,7 +27,6 @@ import io.daos.obj.DaosObject; import io.daos.obj.DaosObjectId; import io.daos.obj.IODataDesc; -import org.apache.spark.SparkConf; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -53,14 +52,14 @@ public class DaosWriterTest { @Test public void testGetLensWithAllEmptyPartitions() { - DaosWriter.WriteConfig writeConfig = DaosShuffleIO.loadWriteConfig(new SparkConf(false)); - DaosWriter.WriteParam param = new DaosWriter.WriteParam(); + DaosWriter.WriterConfig writeConfig = new DaosWriter.WriterConfig(); + DaosWriterSync.WriteParam param = new DaosWriterSync.WriteParam(); int numPart = 10; param.numPartitions(numPart) .shuffleId(1) .mapId(1) .config(writeConfig); - DaosWriter writer = new DaosWriter(param, null, null); + DaosWriterSync writer = new DaosWriterSync(null, param, null); long[] lens = writer.getPartitionLens(numPart); Assert.assertEquals(numPart, lens.length); for (int i = 0; i < numPart; i++) { @@ -81,14 +80,14 @@ public void testGetLensWithPartialEmptyPartitions() throws Exception { Mockito.doNothing().when(daosObject).update(any(IODataDesc.class)); - DaosWriter.WriteConfig writeConfig = DaosShuffleIO.loadWriteConfig(new SparkConf(false)); - DaosWriter.WriteParam param = new DaosWriter.WriteParam(); + DaosWriter.WriterConfig writeConfig = new DaosWriter.WriterConfig(); + DaosWriterSync.WriteParam param = new DaosWriterSync.WriteParam(); int numPart = 10; param.numPartitions(numPart) .shuffleId(1) .mapId(1) .config(writeConfig); - DaosWriter writer = new DaosWriter(param, daosObject, null); + DaosWriterSync writer = new DaosWriterSync(daosObject, param, null); Map expectedLens = new HashMap<>(); Random random = new Random(); for (int i = 0; i < 5; i++) { @@ -133,8 +132,8 @@ public void testWriteTaskFailed() throws Exception { return invoc; }).when(daosObject).update(any(IODataDesc.class)); - DaosWriter.WriteConfig writeConfig = DaosShuffleIO.loadWriteConfig(new SparkConf(false)); - DaosWriter.WriteParam param = new DaosWriter.WriteParam(); + DaosWriter.WriterConfig writeConfig = new DaosWriter.WriterConfig(); + DaosWriterSync.WriteParam param = new DaosWriterSync.WriteParam(); int numPart = 10; param.numPartitions(numPart) .shuffleId(1) @@ -142,8 +141,8 @@ public void testWriteTaskFailed() throws Exception { .config(writeConfig); BoundThreadExecutors executors = new BoundThreadExecutors("read_executors", 1, - new DaosReader.ReadThreadFactory()); - DaosWriter writer = new DaosWriter(param, daosObject, executors.nextExecutor()); + new DaosReaderSync.ReadThreadFactory()); + DaosWriterSync writer = new DaosWriterSync(daosObject, param, executors.nextExecutor()); for (int i = 0; i < numPart; i++) { writer.write(i, new byte[100]); writer.flush(i); diff --git a/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleReaderSuite.scala b/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleReaderSuite.scala index 6a9591ad..98be6b69 100644 --- a/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleReaderSuite.scala +++ b/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleReaderSuite.scala @@ -35,21 +35,22 @@ import org.apache.spark._ import org.apache.spark.internal.config import org.apache.spark.serializer.{JavaSerializer, SerializerManager} import org.apache.spark.shuffle.BaseShuffleHandle +import org.apache.spark.shuffle.daos.DaosReader.ReaderConfig import org.apache.spark.storage.{BlockManagerId, ShuffleBlockId} class DaosShuffleReaderSuite extends SparkFunSuite with LocalSparkContext { override def beforeAll(): Unit = { super.beforeAll() - logInfo("start executors in DaosReader " + classOf[DaosReader]) + logInfo("start executors in DaosReaderSync " + classOf[DaosReaderSync]) MockitoAnnotations.initMocks(this) } private def mockObjectsForSingleDaosCall(reduceId: Int, numMaps: Int, byteOutputStream: ByteArrayOutputStream): (DaosReader, DaosShuffleIO, DaosObject) = { // mock - val daosReader: DaosReader = Mockito.mock(classOf[DaosReader]) val daosObject = Mockito.mock(classOf[DaosObject]) + val daosReader: DaosReaderSync = new DaosReaderSync(daosObject, new ReaderConfig(), null) val shuffleIO = Mockito.mock(classOf[DaosShuffleIO]) val desc = Mockito.mock(classOf[IODataDesc]) @@ -74,8 +75,10 @@ class DaosShuffleReaderSuite extends SparkFunSuite with LocalSparkContext { (DaosReader, DaosShuffleIO, DaosObject) = { // mock val daosObject = Mockito.mock(classOf[DaosObject]) - val daosReader: DaosReader = - if (executors != null) Mockito.spy(new DaosReader(daosObject, executors)) else Mockito.mock(classOf[DaosReader]) + val daosReader: DaosReaderSync = + if (executors != null) Mockito.spy(new DaosReaderSync(daosObject, new DaosReader.ReaderConfig(), + executors.nextExecutor())) + else new DaosReaderSync(daosObject, new ReaderConfig(), null) val shuffleIO = Mockito.mock(classOf[DaosShuffleIO]) val descList = new util.ArrayList[IODataDesc] @@ -177,7 +180,7 @@ class DaosShuffleReaderSuite extends SparkFunSuite with LocalSparkContext { } when(shuffleIO.getDaosReader(shuffleId)).thenReturn(daosReader) - when(daosReader.getObject).thenReturn(daosObject) + // when(daosReader.getObject).thenReturn(daosObject) val shuffleReader = new DaosShuffleReader[Int, Int]( shuffleHandle, @@ -205,7 +208,7 @@ class DaosShuffleReaderSuite extends SparkFunSuite with LocalSparkContext { } test("test reader daos multiple times from other thread") { - val executors = new BoundThreadExecutors("read_executors", 1, new DaosReader.ReadThreadFactory) + val executors = new BoundThreadExecutors("read_executors", 1, new DaosReaderSync.ReadThreadFactory) testRead(7168, 6, false, executors) executors.stop() } From 18d8a4b981353e65e7f5b25534342258855390be Mon Sep 17 00:00:00 2001 From: jiafu zhang Date: Mon, 1 Feb 2021 02:24:37 +0000 Subject: [PATCH 05/12] fix travis warning and unused import in shuffle-hadoop Signed-off-by: jiafu zhang --- .travis.yml | 1 - .../spark/shuffle/sort/RemoteBypassMergeSortShuffleWriter.java | 1 - .../org/apache/spark/shuffle/sort/RemoteUnsafeShuffleWriter.java | 1 - 3 files changed, 3 deletions(-) diff --git a/.travis.yml b/.travis.yml index c19abb3a..15cadf21 100644 --- a/.travis.yml +++ b/.travis.yml @@ -8,5 +8,4 @@ jobs: install: - #empty install step script: - - cd ${TRAVIS_BUILD_DIR}/oap-shuffle/remote-shuffle/ - mvn -q test diff --git a/shuffle-hadoop/src/main/java/org/apache/spark/shuffle/sort/RemoteBypassMergeSortShuffleWriter.java b/shuffle-hadoop/src/main/java/org/apache/spark/shuffle/sort/RemoteBypassMergeSortShuffleWriter.java index 71934d10..8b06b7d2 100644 --- a/shuffle-hadoop/src/main/java/org/apache/spark/shuffle/sort/RemoteBypassMergeSortShuffleWriter.java +++ b/shuffle-hadoop/src/main/java/org/apache/spark/shuffle/sort/RemoteBypassMergeSortShuffleWriter.java @@ -38,7 +38,6 @@ import org.slf4j.LoggerFactory; import org.apache.spark.*; -import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; import org.apache.spark.serializer.Serializer; diff --git a/shuffle-hadoop/src/main/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleWriter.java b/shuffle-hadoop/src/main/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleWriter.java index f87d35d1..0e0ba91c 100644 --- a/shuffle-hadoop/src/main/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleWriter.java +++ b/shuffle-hadoop/src/main/java/org/apache/spark/shuffle/sort/RemoteUnsafeShuffleWriter.java @@ -44,7 +44,6 @@ import org.apache.spark.SparkConf; import org.apache.spark.TaskContext; import org.apache.spark.annotation.Private; -import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.internal.config.package$; import org.apache.spark.io.CompressionCodec; import org.apache.spark.io.CompressionCodec$; From c8aa8065f0accfe6a7b4d4ba174070bad2549687 Mon Sep 17 00:00:00 2001 From: jiafu zhang Date: Mon, 1 Feb 2021 05:41:23 +0000 Subject: [PATCH 06/12] fix travis build issue Signed-off-by: jiafu zhang --- .travis.yml | 2 + .../spark/shuffle/daos/DaosShuffleIOTest.java | 76 ++++++++++--------- .../spark/shuffle/daos/DaosWriterTest.java | 22 ++++++ 3 files changed, 64 insertions(+), 36 deletions(-) diff --git a/.travis.yml b/.travis.yml index 15cadf21..d118e4e2 100644 --- a/.travis.yml +++ b/.travis.yml @@ -5,6 +5,8 @@ jdk: openjdk8 jobs: include: - name: oap-shuffle-remote-shuffle + before_install: + - MAVEN_OPTS="-Xmx2048m" install: - #empty install step script: diff --git a/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleIOTest.java b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleIOTest.java index 9cb77d6e..42135162 100644 --- a/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleIOTest.java +++ b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleIOTest.java @@ -66,49 +66,53 @@ public void testSingleObjectInstanceOpen() throws Exception { UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser("test")); SparkContext sc = new SparkContext("local", "test", testConf); - DaosShuffleIO io = new DaosShuffleIO(testConf); + try { - DaosObjectId id = PowerMockito.mock(DaosObjectId.class); - PowerMockito.whenNew(DaosObjectId.class).withArguments(appId, Long.valueOf(shuffleId)).thenReturn(id); - Mockito.doNothing().when(id).encode(); - Mockito.when(id.isEncoded()).thenReturn(true); - DaosObject daosObject = PowerMockito.mock(DaosObject.class); - DaosObjClient client = PowerMockito.mock(DaosObjClient.class); - Mockito.when(client.getObject(id)).thenReturn(daosObject); + DaosShuffleIO io = new DaosShuffleIO(testConf); - AtomicBoolean open = new AtomicBoolean(false); - Mockito.when(daosObject.isOpen()).then(invocationOnMock -> - open.get() - ); - Mockito.doAnswer(invocationOnMock -> { - open.compareAndSet(false, true); - return invocationOnMock; - }).when(daosObject).open(); - clientField.set(io.getIoManager(), client); + DaosObjectId id = PowerMockito.mock(DaosObjectId.class); + PowerMockito.whenNew(DaosObjectId.class).withArguments(appId, Long.valueOf(shuffleId)).thenReturn(id); + Mockito.doNothing().when(id).encode(); + Mockito.when(id.isEncoded()).thenReturn(true); + DaosObject daosObject = PowerMockito.mock(DaosObject.class); + DaosObjClient client = PowerMockito.mock(DaosObjClient.class); + Mockito.when(client.getObject(id)).thenReturn(daosObject); - int numThreads = 50; - ExecutorService es = Executors.newFixedThreadPool(numThreads); - AtomicInteger count = new AtomicInteger(0); + AtomicBoolean open = new AtomicBoolean(false); + Mockito.when(daosObject.isOpen()).then(invocationOnMock -> + open.get() + ); + Mockito.doAnswer(invocationOnMock -> { + open.compareAndSet(false, true); + return invocationOnMock; + }).when(daosObject).open(); + clientField.set(io.getIoManager(), client); - Runnable r = () -> { - try { - DaosReader reader = io.getDaosReader(shuffleId); - if (reader.getObject() == daosObject && reader.getObject().isOpen()) { - count.incrementAndGet(); + int numThreads = 50; + ExecutorService es = Executors.newFixedThreadPool(numThreads); + AtomicInteger count = new AtomicInteger(0); + + Runnable r = () -> { + try { + DaosReader reader = io.getDaosReader(shuffleId); + if (reader.getObject() == daosObject && reader.getObject().isOpen()) { + count.incrementAndGet(); + } + } catch (Exception e) { + e.printStackTrace(); } - } catch (Exception e) { - e.printStackTrace(); - } - }; + }; - for (int i = 0; i < numThreads; i++) { - es.submit(r); - } + for (int i = 0; i < numThreads; i++) { + es.submit(r); + } - es.shutdown(); - es.awaitTermination(5, TimeUnit.SECONDS); + es.shutdown(); + es.awaitTermination(5, TimeUnit.SECONDS); - Assert.assertEquals(50, count.intValue()); - sc.stop(); + Assert.assertEquals(50, count.intValue()); + } finally { + sc.stop(); + } } } diff --git a/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosWriterTest.java b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosWriterTest.java index 3f5bc2fb..f25ac74d 100644 --- a/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosWriterTest.java +++ b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosWriterTest.java @@ -27,7 +27,12 @@ import io.daos.obj.DaosObject; import io.daos.obj.DaosObjectId; import io.daos.obj.IODataDesc; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; +import org.junit.AfterClass; import org.junit.Assert; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mockito; @@ -50,6 +55,16 @@ @SuppressStaticInitializationFor("io.daos.obj.DaosObjClient") public class DaosWriterTest { + private static SparkConf testConf = new SparkConf(false); + + private static SparkContext sc; + + @BeforeClass + public static void initialize() { + UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser("test")); + sc = new SparkContext("local", "test", testConf); + } + @Test public void testGetLensWithAllEmptyPartitions() { DaosWriter.WriterConfig writeConfig = new DaosWriter.WriterConfig(); @@ -152,4 +167,11 @@ public void testWriteTaskFailed() throws Exception { executors.stop(); } + + @AfterClass + public static void teardown() { + if (sc != null) { + sc.stop(); + } + } } From 886e19dd52ce2ae49b4c9d308520d784051f1ecf Mon Sep 17 00:00:00 2001 From: jiafu zhang Date: Mon, 1 Feb 2021 07:31:10 +0000 Subject: [PATCH 07/12] fix travis build Signed-off-by: jiafu zhang --- .travis.yml | 4 ++-- .../apache/spark/shuffle/daos/DaosShuffleInputStreamTest.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.travis.yml b/.travis.yml index d118e4e2..a9595b6e 100644 --- a/.travis.yml +++ b/.travis.yml @@ -6,8 +6,8 @@ jobs: include: - name: oap-shuffle-remote-shuffle before_install: - - MAVEN_OPTS="-Xmx2048m" + - MAVEN_OPTS="-Xmx2948m" install: - #empty install step script: - - mvn -q test + - mvn clean install diff --git a/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleInputStreamTest.java b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleInputStreamTest.java index 6c41d92b..35be8d53 100644 --- a/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleInputStreamTest.java +++ b/shuffle-daos/src/test/java/org/apache/spark/shuffle/daos/DaosShuffleInputStreamTest.java @@ -129,7 +129,7 @@ public void testReadFromOtherThreadCancelMultipleTimesLongWait() throws Exceptio Map maps = new HashMap<>(); maps.put("2", new AtomicInteger(0)); maps.put("4", new AtomicInteger(0)); - readFromOtherThreadCancelMultipleTimes(maps, 400); + readFromOtherThreadCancelMultipleTimes(maps, 800); } @Test From d9f8c41b00203f4e32b48a6fa2a83fa7427cefba Mon Sep 17 00:00:00 2001 From: jiafu zhang Date: Mon, 1 Feb 2021 07:41:34 +0000 Subject: [PATCH 08/12] maven quite build Signed-off-by: jiafu zhang --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index a9595b6e..0ef13721 100644 --- a/.travis.yml +++ b/.travis.yml @@ -10,4 +10,4 @@ jobs: install: - #empty install step script: - - mvn clean install + - mvn -q clean install From 75680816a16d58c63f543053fededb23c251298e Mon Sep 17 00:00:00 2001 From: jiafu zhang Date: Mon, 1 Feb 2021 08:12:19 +0000 Subject: [PATCH 09/12] fix traviswq outofmemroy issue Signed-off-by: jiafu zhang --- .travis.yml | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index 0ef13721..8c366280 100644 --- a/.travis.yml +++ b/.travis.yml @@ -6,7 +6,7 @@ jobs: include: - name: oap-shuffle-remote-shuffle before_install: - - MAVEN_OPTS="-Xmx2948m" + - echo "MAVEN_OPTS='-Xmx2948m'" > ~/.mavenrc install: - #empty install step script: diff --git a/pom.xml b/pom.xml index 79d831e1..c5ea1e29 100644 --- a/pom.xml +++ b/pom.xml @@ -259,8 +259,8 @@ - shuffle-hadoop shuffle-daos + shuffle-hadoop From 241fbbe1f9c9e168b71f461bd580aaba5d9da261 Mon Sep 17 00:00:00 2001 From: jiafu zhang Date: Mon, 1 Feb 2021 09:15:37 +0000 Subject: [PATCH 10/12] fix travis outofmemory issue Signed-off-by: jiafu zhang --- .travis.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 8c366280..d5354d16 100644 --- a/.travis.yml +++ b/.travis.yml @@ -6,7 +6,9 @@ jobs: include: - name: oap-shuffle-remote-shuffle before_install: - - echo "MAVEN_OPTS='-Xmx2948m'" > ~/.mavenrc + - sudo rm /etc/mavenrc + - export M2_HOME=/usr/local/maven + - export MAVEN_OPTS="-Dmaven.repo.local=$HOME/.m2/repository -Xmx3072m" install: - #empty install step script: From 4fcca320d01581c45a594726fb00e290f34f0535 Mon Sep 17 00:00:00 2001 From: jiafu zhang Date: Mon, 1 Feb 2021 09:32:49 +0000 Subject: [PATCH 11/12] fix travis outofmemory issue Signed-off-by: jiafu zhang --- .travis.yml | 4 ---- shuffle-daos/pom.xml | 3 +++ 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/.travis.yml b/.travis.yml index d5354d16..e3bafd2d 100644 --- a/.travis.yml +++ b/.travis.yml @@ -5,10 +5,6 @@ jdk: openjdk8 jobs: include: - name: oap-shuffle-remote-shuffle - before_install: - - sudo rm /etc/mavenrc - - export M2_HOME=/usr/local/maven - - export MAVEN_OPTS="-Dmaven.repo.local=$HOME/.m2/repository -Xmx3072m" install: - #empty install step script: diff --git a/shuffle-daos/pom.xml b/shuffle-daos/pom.xml index 91588e6f..9e4f28a8 100644 --- a/shuffle-daos/pom.xml +++ b/shuffle-daos/pom.xml @@ -121,6 +121,9 @@ org.scalatest scalatest-maven-plugin + + -Xmx2048m + test From bf0b8efab8dc99440a92dc048ffe2bd7e0096812 Mon Sep 17 00:00:00 2001 From: jiafu zhang Date: Mon, 1 Feb 2021 09:49:13 +0000 Subject: [PATCH 12/12] reduce data size to avoid outofmemory in test code Signed-off-by: jiafu zhang --- .../apache/spark/shuffle/daos/DaosShuffleReaderSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleReaderSuite.scala b/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleReaderSuite.scala index 98be6b69..a6975a2b 100644 --- a/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleReaderSuite.scala +++ b/shuffle-daos/src/test/scala/org/apache/spark/shuffle/daos/DaosShuffleReaderSuite.scala @@ -204,12 +204,12 @@ class DaosShuffleReaderSuite extends SparkFunSuite with LocalSparkContext { } test("test reader daos multiple times") { - testRead(7168, 4, false) + testRead(1024, 4, false) } test("test reader daos multiple times from other thread") { val executors = new BoundThreadExecutors("read_executors", 1, new DaosReaderSync.ReadThreadFactory) - testRead(7168, 6, false, executors) + testRead(1024, 6, false, executors) executors.stop() } }