├── .gitignore ├── .travis.yml ├── CONTRIBUTING.md ├── LICENSE ├── pom.xml └── src ├── main ├── java │ ├── com │ │ └── facebook │ │ │ └── presto │ │ │ └── hadoop │ │ │ ├── FileSystemFactory.java │ │ │ ├── HadoopFileSystemCache.java │ │ │ ├── HadoopNative.java │ │ │ └── TextLineLengthLimitExceededException.java │ └── org │ │ └── apache │ │ └── hadoop │ │ ├── fs │ │ ├── FinalizerService.java │ │ ├── InputStreamWrapper.java │ │ ├── OutputStreamWrapper.java │ │ ├── PrestoFileSystemCache.java │ │ └── PrestoFilterFileSystemWrapper.java │ │ ├── security │ │ └── UserGroupInformationShim.java │ │ └── util │ │ └── LineReader.java └── resources │ ├── core-site.xml │ └── nativelib │ ├── Linux-aarch64 │ ├── libhadoop.so │ └── libsnappy.so │ ├── Linux-amd64 │ ├── libhadoop.so │ └── libsnappy.so │ ├── Linux-ppc64le │ ├── libhadoop.so │ └── libsnappy.so │ ├── Mac_OS_X-aarch64 │ ├── libhadoop.dylib │ └── libsnappy.dylib │ └── Mac_OS_X-x86_64 │ ├── libhadoop.dylib │ └── libsnappy.dylib └── test └── java └── com └── facebook └── presto └── hadoop ├── TestHadoopFileSystemCache.java ├── TestHadoopNative.java └── TestLineReader.java /.gitignore: -------------------------------------------------------------------------------- 1 | *.iml 2 | *.ipr 3 | *.iws 4 | target/ 5 | /var 6 | pom.xml.versionsBackup 7 | test-output/ 8 | /atlassian-ide-plugin.xml 9 | .idea 10 | .DS_Store 11 | .classpath 12 | .settings 13 | .project 14 | temp-testng-customsuite.xml 15 | test-output 16 | .externalToolBuilders 17 | *~ 18 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: java 2 | -------------------------------------------------------------------------------- /CONTRIBUTING.md: -------------------------------------------------------------------------------- 1 | # Contributing to presto-hadoop-apache2 2 | We want to make contributing to this project as easy and transparent as 3 | possible. 4 | 5 | ## Pull Requests 6 | We actively welcome your pull requests. 7 | 8 | 1. Fork the repo and create your branch from `master`. 9 | 2. If you've added code that should be tested, add tests. 10 | 3. If you've changed APIs, update the documentation. 11 | 4. Ensure the test suite passes. 12 | 5. Make sure your code lints. 13 | 6. If you haven't already, complete the Contributor License Agreement ("CLA"). 14 | 15 | ## Contributor License Agreement ("CLA") 16 | In order to accept your pull request, we need you to submit a CLA. You only need 17 | to do this once to work on any of Facebook's open source projects. 18 | 19 | Complete your CLA here: 20 | 21 | ## Issues 22 | We use GitHub issues to track public bugs. Please ensure your description is 23 | clear and has sufficient instructions to be able to reproduce the issue. 24 | 25 | Facebook has a [bounty program](https://www.facebook.com/whitehat/) for the safe 26 | disclosure of security bugs. In those cases, please go through the process 27 | outlined on that page and do not file a public issue. 28 | 29 | ## License 30 | By contributing to presto-hadoop-apache2, you agree that your contributions will be licensed 31 | under the LICENSE file in the root directory of this source tree. 32 | 33 | ## Maintainers 34 | In addition to PrestoDB committers, this project is also maintained by the individuals below, 35 | who have committer rights to this repository: 36 | 37 | * [Ariel Weisberg](https://github.com/aweisberg) 38 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | 2 | Apache License 3 | Version 2.0, January 2004 4 | http://www.apache.org/licenses/ 5 | 6 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 7 | 8 | 1. Definitions. 9 | 10 | "License" shall mean the terms and conditions for use, reproduction, 11 | and distribution as defined by Sections 1 through 9 of this document. 12 | 13 | "Licensor" shall mean the copyright owner or entity authorized by 14 | the copyright owner that is granting the License. 15 | 16 | "Legal Entity" shall mean the union of the acting entity and all 17 | other entities that control, are controlled by, or are under common 18 | control with that entity. For the purposes of this definition, 19 | "control" means (i) the power, direct or indirect, to cause the 20 | direction or management of such entity, whether by contract or 21 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 22 | outstanding shares, or (iii) beneficial ownership of such entity. 23 | 24 | "You" (or "Your") shall mean an individual or Legal Entity 25 | exercising permissions granted by this License. 26 | 27 | "Source" form shall mean the preferred form for making modifications, 28 | including but not limited to software source code, documentation 29 | source, and configuration files. 30 | 31 | "Object" form shall mean any form resulting from mechanical 32 | transformation or translation of a Source form, including but 33 | not limited to compiled object code, generated documentation, 34 | and conversions to other media types. 35 | 36 | "Work" shall mean the work of authorship, whether in Source or 37 | Object form, made available under the License, as indicated by a 38 | copyright notice that is included in or attached to the work 39 | (an example is provided in the Appendix below). 40 | 41 | "Derivative Works" shall mean any work, whether in Source or Object 42 | form, that is based on (or derived from) the Work and for which the 43 | editorial revisions, annotations, elaborations, or other modifications 44 | represent, as a whole, an original work of authorship. For the purposes 45 | of this License, Derivative Works shall not include works that remain 46 | separable from, or merely link (or bind by name) to the interfaces of, 47 | the Work and Derivative Works thereof. 48 | 49 | "Contribution" shall mean any work of authorship, including 50 | the original version of the Work and any modifications or additions 51 | to that Work or Derivative Works thereof, that is intentionally 52 | submitted to Licensor for inclusion in the Work by the copyright owner 53 | or by an individual or Legal Entity authorized to submit on behalf of 54 | the copyright owner. For the purposes of this definition, "submitted" 55 | means any form of electronic, verbal, or written communication sent 56 | to the Licensor or its representatives, including but not limited to 57 | communication on electronic mailing lists, source code control systems, 58 | and issue tracking systems that are managed by, or on behalf of, the 59 | Licensor for the purpose of discussing and improving the Work, but 60 | excluding communication that is conspicuously marked or otherwise 61 | designated in writing by the copyright owner as "Not a Contribution." 62 | 63 | "Contributor" shall mean Licensor and any individual or Legal Entity 64 | on behalf of whom a Contribution has been received by Licensor and 65 | subsequently incorporated within the Work. 66 | 67 | 2. Grant of Copyright License. Subject to the terms and conditions of 68 | this License, each Contributor hereby grants to You a perpetual, 69 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 70 | copyright license to reproduce, prepare Derivative Works of, 71 | publicly display, publicly perform, sublicense, and distribute the 72 | Work and such Derivative Works in Source or Object form. 73 | 74 | 3. Grant of Patent License. Subject to the terms and conditions of 75 | this License, each Contributor hereby grants to You a perpetual, 76 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 77 | (except as stated in this section) patent license to make, have made, 78 | use, offer to sell, sell, import, and otherwise transfer the Work, 79 | where such license applies only to those patent claims licensable 80 | by such Contributor that are necessarily infringed by their 81 | Contribution(s) alone or by combination of their Contribution(s) 82 | with the Work to which such Contribution(s) was submitted. If You 83 | institute patent litigation against any entity (including a 84 | cross-claim or counterclaim in a lawsuit) alleging that the Work 85 | or a Contribution incorporated within the Work constitutes direct 86 | or contributory patent infringement, then any patent licenses 87 | granted to You under this License for that Work shall terminate 88 | as of the date such litigation is filed. 89 | 90 | 4. Redistribution. You may reproduce and distribute copies of the 91 | Work or Derivative Works thereof in any medium, with or without 92 | modifications, and in Source or Object form, provided that You 93 | meet the following conditions: 94 | 95 | (a) You must give any other recipients of the Work or 96 | Derivative Works a copy of this License; and 97 | 98 | (b) You must cause any modified files to carry prominent notices 99 | stating that You changed the files; and 100 | 101 | (c) You must retain, in the Source form of any Derivative Works 102 | that You distribute, all copyright, patent, trademark, and 103 | attribution notices from the Source form of the Work, 104 | excluding those notices that do not pertain to any part of 105 | the Derivative Works; and 106 | 107 | (d) If the Work includes a "NOTICE" text file as part of its 108 | distribution, then any Derivative Works that You distribute must 109 | include a readable copy of the attribution notices contained 110 | within such NOTICE file, excluding those notices that do not 111 | pertain to any part of the Derivative Works, in at least one 112 | of the following places: within a NOTICE text file distributed 113 | as part of the Derivative Works; within the Source form or 114 | documentation, if provided along with the Derivative Works; or, 115 | within a display generated by the Derivative Works, if and 116 | wherever such third-party notices normally appear. The contents 117 | of the NOTICE file are for informational purposes only and 118 | do not modify the License. You may add Your own attribution 119 | notices within Derivative Works that You distribute, alongside 120 | or as an addendum to the NOTICE text from the Work, provided 121 | that such additional attribution notices cannot be construed 122 | as modifying the License. 123 | 124 | You may add Your own copyright statement to Your modifications and 125 | may provide additional or different license terms and conditions 126 | for use, reproduction, or distribution of Your modifications, or 127 | for any such Derivative Works as a whole, provided Your use, 128 | reproduction, and distribution of the Work otherwise complies with 129 | the conditions stated in this License. 130 | 131 | 5. Submission of Contributions. Unless You explicitly state otherwise, 132 | any Contribution intentionally submitted for inclusion in the Work 133 | by You to the Licensor shall be under the terms and conditions of 134 | this License, without any additional terms or conditions. 135 | Notwithstanding the above, nothing herein shall supersede or modify 136 | the terms of any separate license agreement you may have executed 137 | with Licensor regarding such Contributions. 138 | 139 | 6. Trademarks. This License does not grant permission to use the trade 140 | names, trademarks, service marks, or product names of the Licensor, 141 | except as required for reasonable and customary use in describing the 142 | origin of the Work and reproducing the content of the NOTICE file. 143 | 144 | 7. Disclaimer of Warranty. Unless required by applicable law or 145 | agreed to in writing, Licensor provides the Work (and each 146 | Contributor provides its Contributions) on an "AS IS" BASIS, 147 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 148 | implied, including, without limitation, any warranties or conditions 149 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 150 | PARTICULAR PURPOSE. You are solely responsible for determining the 151 | appropriateness of using or redistributing the Work and assume any 152 | risks associated with Your exercise of permissions under this License. 153 | 154 | 8. Limitation of Liability. In no event and under no legal theory, 155 | whether in tort (including negligence), contract, or otherwise, 156 | unless required by applicable law (such as deliberate and grossly 157 | negligent acts) or agreed to in writing, shall any Contributor be 158 | liable to You for damages, including any direct, indirect, special, 159 | incidental, or consequential damages of any character arising as a 160 | result of this License or out of the use or inability to use the 161 | Work (including but not limited to damages for loss of goodwill, 162 | work stoppage, computer failure or malfunction, or any and all 163 | other commercial damages or losses), even if such Contributor 164 | has been advised of the possibility of such damages. 165 | 166 | 9. Accepting Warranty or Additional Liability. While redistributing 167 | the Work or Derivative Works thereof, You may choose to offer, 168 | and charge a fee for, acceptance of support, warranty, indemnity, 169 | or other liability obligations and/or rights consistent with this 170 | License. However, in accepting such obligations, You may act only 171 | on Your own behalf and on Your sole responsibility, not on behalf 172 | of any other Contributor, and only if You agree to indemnify, 173 | defend, and hold each Contributor harmless for any liability 174 | incurred by, or claims asserted against, such Contributor by reason 175 | of your accepting any such warranty or additional liability. 176 | 177 | END OF TERMS AND CONDITIONS 178 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4.0.0 4 | 5 | 6 | com.facebook.airlift 7 | airbase 8 | 98 9 | 10 | 11 | com.facebook.presto.hadoop 12 | hadoop-apache2 13 | 2.7.4-13-SNAPSHOT 14 | 15 | hadoop-apache2 16 | Shaded version of Apache Hadoop 2.x for Presto 17 | https://github.com/facebook/presto-hadoop-apache2 18 | 19 | 2013 20 | 21 | 22 | 23 | Apache License 2.0 24 | http://www.apache.org/licenses/LICENSE-2.0 25 | repo 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | scm:git:git://github.com/facebook/presto-hadoop-apache2.git 35 | scm:git:git://github.com/facebook/presto-hadoop-apache2.git 36 | https://github.com/facebook/presto-hadoop-apache2 37 | HEAD 38 | 39 | 40 | 41 | UTF-8 42 | 1.7 43 | com.facebook.presto.hadoop.\$internal 44 | 1.7.13 45 | 2.7.4 46 | 47 | true 48 | true 49 | true 50 | true 51 | true 52 | true 53 | 54 | -missing 55 | 56 | 57 | 58 | 59 | org.apache.hadoop 60 | hadoop-common 61 | ${dep.hadoop.version} 62 | 63 | 64 | org.apache.hadoop 65 | hadoop-mapreduce-client-app 66 | 67 | 68 | com.google.errorprone 69 | error_prone_annotations 70 | 71 | 72 | com.google.j2objc 73 | j2objc-annotations 74 | 75 | 76 | org.checkerframework 77 | checker-qual 78 | 79 | 80 | org.codehaus.mojo 81 | animal-sniffer-annotations 82 | 83 | 84 | org.apache.hadoop 85 | hadoop-mapreduce-client-jobclient 86 | 87 | 88 | org.apache.hadoop 89 | hadoop-yarn-api 90 | 91 | 92 | org.apache.hadoop 93 | hadoop-yarn-common 94 | 95 | 96 | org.apache.zookeeper 97 | zookeeper 98 | 99 | 100 | org.apache.curator 101 | curator-framework 102 | 103 | 104 | org.apache.curator 105 | curator-client 106 | 107 | 108 | org.apache.curator 109 | curator-recipes 110 | 111 | 112 | org.apache.directory.server 113 | apacheds-kerberos-codec 114 | 115 | 116 | org.apache.commons 117 | commons-compress 118 | 119 | 120 | org.apache.commons 121 | commons-math 122 | 123 | 124 | org.apache.commons 125 | commons-math3 126 | 127 | 128 | commons-beanutils 129 | commons-beanutils 130 | 131 | 132 | commons-beanutils 133 | commons-beanutils-core 134 | 135 | 136 | commons-digester 137 | commons-digester 138 | 139 | 140 | commons-el 141 | commons-el 142 | 143 | 144 | commons-logging 145 | commons-logging 146 | 147 | 148 | com.sun.jersey 149 | jersey-core 150 | 151 | 152 | com.sun.jersey 153 | jersey-server 154 | 155 | 156 | com.sun.jersey 157 | jersey-json 158 | 159 | 160 | tomcat 161 | jasper-compiler 162 | 163 | 164 | tomcat 165 | jasper-runtime 166 | 167 | 168 | org.mortbay.jetty 169 | jetty 170 | 171 | 172 | javax.servlet 173 | servlet-api 174 | 175 | 176 | javax.servlet.jsp 177 | jsp-api 178 | 179 | 180 | xmlenc 181 | xmlenc 182 | 183 | 184 | com.jcraft 185 | jsch 186 | 187 | 188 | jdk.tools 189 | jdk.tools 190 | 191 | 192 | com.thoughtworks.paranamer 193 | paranamer 194 | 195 | 196 | org.xerial.snappy 197 | snappy-java 198 | 199 | 200 | com.google.code.findbugs 201 | jsr305 202 | 203 | 204 | net.java.dev.jets3t 205 | jets3t 206 | 207 | 208 | log4j 209 | log4j 210 | 211 | 212 | org.slf4j 213 | slf4j-log4j12 214 | 215 | 216 | 217 | 218 | 219 | org.apache.hadoop 220 | hadoop-hdfs 221 | ${dep.hadoop.version} 222 | runtime 223 | 224 | 225 | commons-codec 226 | commons-codec 227 | 228 | 229 | commons-daemon 230 | commons-daemon 231 | 232 | 233 | commons-el 234 | commons-el 235 | 236 | 237 | commons-logging 238 | commons-logging 239 | 240 | 241 | org.fusesource.leveldbjni 242 | leveldbjni-all 243 | 244 | 245 | com.sun.jersey 246 | jersey-core 247 | 248 | 249 | tomcat 250 | jasper-runtime 251 | 252 | 253 | com.sun.jersey 254 | jersey-server 255 | 256 | 257 | org.mortbay.jetty 258 | jetty 259 | 260 | 261 | org.mortbay.jetty 262 | jetty-util 263 | 264 | 265 | javax.servlet 266 | servlet-api 267 | 268 | 269 | javax.servlet.jsp 270 | jsp-api 271 | 272 | 273 | xerces 274 | xercesImpl 275 | 276 | 277 | xmlenc 278 | xmlenc 279 | 280 | 281 | log4j 282 | log4j 283 | 284 | 285 | io.netty 286 | netty 287 | 288 | 289 | 290 | 291 | 292 | org.apache.hadoop 293 | hadoop-mapreduce-client-core 294 | ${dep.hadoop.version} 295 | runtime 296 | 297 | 298 | org.apache.hadoop 299 | hadoop-yarn-common 300 | 301 | 302 | com.google.inject.extensions 303 | guice-servlet 304 | 305 | 306 | io.netty 307 | netty 308 | 309 | 310 | org.jboss.netty 311 | netty 312 | 313 | 314 | org.slf4j 315 | slf4j-log4j12 316 | 317 | 318 | 319 | 320 | 321 | org.apache.hadoop 322 | hadoop-azure 323 | ${dep.hadoop.version} 324 | runtime 325 | 326 | 327 | org.apache.httpcomponents 328 | httpclient 329 | 330 | 331 | 332 | 333 | 334 | 335 | org.slf4j 336 | jcl-over-slf4j 337 | ${dep.slf4j.version} 338 | 339 | 340 | 341 | org.slf4j 342 | log4j-over-slf4j 343 | ${dep.slf4j.version} 344 | runtime 345 | 346 | 347 | 348 | org.slf4j 349 | slf4j-api 350 | ${dep.slf4j.version} 351 | runtime 352 | 353 | 354 | 355 | org.slf4j 356 | slf4j-jdk14 357 | ${dep.slf4j.version} 358 | runtime 359 | 360 | 361 | 362 | 363 | org.testng 364 | testng 365 | 6.2.1 366 | test 367 | 368 | 369 | 370 | 371 | 372 | 373 | org.apache.maven.plugins 374 | maven-compiler-plugin 375 | 3.1 376 | 377 | ${project.build.targetJdk} 378 | ${project.build.targetJdk} 379 | 380 | 381 | 382 | 383 | org.apache.maven.plugins 384 | maven-surefire-plugin 385 | 2.12.4 386 | 387 | 388 | 389 | org.apache.maven.plugins 390 | maven-source-plugin 391 | 2.2.1 392 | 393 | 394 | attach-sources 395 | 396 | jar-no-fork 397 | 398 | 399 | 400 | 401 | 402 | 403 | org.apache.maven.plugins 404 | maven-shade-plugin 405 | 2.4.2 406 | 407 | 408 | package 409 | 410 | shade 411 | 412 | 413 | true 414 | true 415 | ${project.build.directory}/pom.xml 416 | true 417 | 418 | 419 | 420 | 421 | 422 | org.iq80.leveldb 423 | ${shadeBase}.org.iq80.leveldb 424 | 425 | 426 | org.apache.http 427 | ${shadeBase}.org.apache.http 428 | 429 | 430 | io.netty 431 | ${shadeBase}.io.netty 432 | 433 | 434 | com.microsoft.azure 435 | ${shadeBase}.com.microsoft.azure 436 | 437 | 438 | org.apache.avro 439 | ${shadeBase}.org.apache.avro 440 | 441 | 442 | org.apache.commons 443 | ${shadeBase}.org.apache.commons 444 | 445 | 446 | com.google.common 447 | ${shadeBase}.com.google.common 448 | 449 | 450 | com.google.protobuf 451 | ${shadeBase}.com.google.protobuf 452 | 453 | 454 | com.google.gson 455 | ${shadeBase}.com.google.gson 456 | 457 | 458 | com.fasterxml.jackson.core 459 | ${shadeBase}.com.fasterxml.jackson.core 460 | 461 | 462 | org.codehaus.jackson 463 | ${shadeBase}.org.codehaus.jackson 464 | 465 | 466 | org.mortbay 467 | ${shadeBase}.org.mortbay 468 | 469 | 470 | org.apache.log4j 471 | ${shadeBase}.org.apache.log4j 472 | 473 | 474 | org.slf4j 475 | ${shadeBase}.org.slf4j 476 | 477 | 478 | com.google.thirdparty.publicsuffix 479 | ${shadeBase}.com.google.thirdparty.publicsuffix 480 | 481 | 482 | 483 | 484 | *:* 485 | 486 | META-INF/maven/** 487 | META-INF/*.SF 488 | META-INF/*.DSA 489 | META-INF/*.RSA 490 | 491 | 492 | 493 | org.slf4j:* 494 | 495 | META-INF/services/** 496 | 497 | 498 | 499 | io.netty:netty-all 500 | 501 | META-INF/io.netty.versions.properties 502 | 503 | 504 | 505 | com.fasterxml.jackson.*:* 506 | 507 | META-INF/services/** 508 | 509 | 510 | 511 | commons-configuration:commons-configuration 512 | 513 | *.xml 514 | *.dtd 515 | 516 | 517 | 518 | commons-httpclient:commons-httpclient 519 | 520 | META-INF/README.txt 521 | 522 | 523 | 524 | org.apache.htrace:htrace-core 525 | 526 | META-INF/services/commons-logging/** 527 | META-INF/services/com.fasterxml.jackson.* 528 | 529 | 530 | 531 | org.apache.hadoop:hadoop-common 532 | 533 | *.h 534 | org/apache/hadoop/util/LineReader.class 535 | 536 | 537 | 538 | org.apache.hadoop:hadoop-mapreduce-client-core 539 | 540 | mapred-default.xml.orig 541 | 542 | 543 | 544 | ${project.groupId}:${project.artifactId} 545 | 546 | ** 547 | 548 | 549 | 550 | 551 | 552 | 553 | 554 | 555 | 556 | org.apache.maven.plugins 557 | maven-release-plugin 558 | 559 | clean verify -DskipTests 560 | 561 | 562 | 563 | 564 | org.sonatype.plugins 565 | nexus-staging-maven-plugin 566 | ${dep.nexus-staging-plugin.version} 567 | true 568 | 569 | ossrh 570 | https://oss.sonatype.org/ 571 | 572 | 573 | 574 | 575 | 576 | 577 | 578 | deploy-to-ossrh 579 | 580 | 581 | 582 | org.sonatype.plugins 583 | nexus-staging-maven-plugin 584 | 585 | 586 | 587 | 588 | 589 | 590 | -------------------------------------------------------------------------------- /src/main/java/com/facebook/presto/hadoop/FileSystemFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package com.facebook.presto.hadoop; 15 | 16 | import org.apache.hadoop.fs.FileSystem; 17 | 18 | import java.net.URI; 19 | 20 | public interface FileSystemFactory 21 | { 22 | FileSystem createFileSystem(URI uri); 23 | } 24 | -------------------------------------------------------------------------------- /src/main/java/com/facebook/presto/hadoop/HadoopFileSystemCache.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package com.facebook.presto.hadoop; 15 | 16 | import org.apache.hadoop.fs.FileSystem; 17 | import org.apache.hadoop.fs.PrestoFileSystemCache; 18 | 19 | import java.lang.reflect.Field; 20 | import java.lang.reflect.Modifier; 21 | 22 | public final class HadoopFileSystemCache 23 | { 24 | private static PrestoFileSystemCache cache; 25 | 26 | private HadoopFileSystemCache() {} 27 | 28 | public static synchronized void initialize() 29 | { 30 | if (cache == null) { 31 | cache = setFinalStatic(FileSystem.class, "CACHE", new PrestoFileSystemCache()); 32 | } 33 | } 34 | 35 | private static T setFinalStatic(Class clazz, String name, T value) 36 | { 37 | try { 38 | Field field = clazz.getDeclaredField(name); 39 | field.setAccessible(true); 40 | 41 | Field modifiersField = Field.class.getDeclaredField("modifiers"); 42 | modifiersField.setAccessible(true); 43 | modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL); 44 | 45 | field.set(null, value); 46 | 47 | return value; 48 | } 49 | catch (ReflectiveOperationException e) { 50 | throw new AssertionError(e); 51 | } 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /src/main/java/com/facebook/presto/hadoop/HadoopNative.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package com.facebook.presto.hadoop; 15 | 16 | import org.apache.hadoop.conf.Configuration; 17 | import org.apache.hadoop.io.compress.CompressionCodec; 18 | import org.apache.hadoop.io.compress.CompressionCodecFactory; 19 | import org.apache.hadoop.util.NativeCodeLoader; 20 | 21 | import java.io.File; 22 | import java.io.IOException; 23 | import java.io.InputStream; 24 | import java.lang.reflect.Field; 25 | import java.net.URL; 26 | import java.nio.file.Files; 27 | import java.nio.file.StandardCopyOption; 28 | 29 | import static org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses; 30 | 31 | public final class HadoopNative 32 | { 33 | private static boolean loaded; 34 | private static Throwable error; 35 | 36 | private HadoopNative() {} 37 | 38 | public static synchronized void requireHadoopNative() 39 | { 40 | if (loaded) { 41 | return; 42 | } 43 | if (error != null) { 44 | throw new RuntimeException("failed to load Hadoop native library", error); 45 | } 46 | try { 47 | loadLibrary("hadoop"); 48 | loadLibrary("snappy"); 49 | setStatic(NativeCodeLoader.class.getDeclaredField("nativeCodeLoaded"), true); 50 | 51 | // verify that all configured codec classes can be loaded 52 | loadAllCodecs(); 53 | 54 | loaded = true; 55 | } 56 | catch (Throwable t) { 57 | error = t; 58 | throw new RuntimeException("failed to load Hadoop native library", error); 59 | } 60 | } 61 | 62 | private static void loadAllCodecs() 63 | { 64 | Configuration conf = new Configuration(); 65 | CompressionCodecFactory factory = new CompressionCodecFactory(conf); 66 | for (Class clazz : getCodecClasses(conf)) { 67 | CompressionCodec codec = factory.getCodecByClassName(clazz.getName()); 68 | if (codec == null) { 69 | throw new RuntimeException("failed to load codec: " + clazz.getName()); 70 | } 71 | codec.getDecompressorType(); 72 | } 73 | } 74 | 75 | private static void setStatic(Field field, Object value) 76 | throws IllegalAccessException 77 | { 78 | field.setAccessible(true); 79 | field.set(null, value); 80 | } 81 | 82 | private static void loadLibrary(String name) 83 | throws IOException 84 | { 85 | String libraryPath = getLibraryPath(name); 86 | URL url = HadoopNative.class.getResource(libraryPath); 87 | if (url == null) { 88 | throw new RuntimeException("library not found: " + libraryPath); 89 | } 90 | 91 | File file = File.createTempFile(name, null); 92 | file.deleteOnExit(); 93 | try (InputStream in = url.openStream()) { 94 | Files.copy(in, file.toPath(), StandardCopyOption.REPLACE_EXISTING); 95 | } 96 | 97 | System.load(file.getAbsolutePath()); 98 | } 99 | 100 | private static String getLibraryPath(String name) 101 | { 102 | return "/nativelib/" + getPlatform() + "/" + System.mapLibraryName(name); 103 | } 104 | 105 | private static String getPlatform() 106 | { 107 | String name = System.getProperty("os.name"); 108 | String arch = System.getProperty("os.arch"); 109 | return (name + "-" + arch).replace(' ', '_'); 110 | } 111 | } 112 | -------------------------------------------------------------------------------- /src/main/java/com/facebook/presto/hadoop/TextLineLengthLimitExceededException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package com.facebook.presto.hadoop; 15 | 16 | import java.io.IOException; 17 | 18 | public class TextLineLengthLimitExceededException 19 | extends IOException 20 | { 21 | public TextLineLengthLimitExceededException(String message) 22 | { 23 | super(message); 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /src/main/java/org/apache/hadoop/fs/FinalizerService.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package org.apache.hadoop.fs; 15 | 16 | import com.google.common.collect.Sets; 17 | import org.apache.commons.logging.Log; 18 | import org.apache.commons.logging.LogFactory; 19 | 20 | import java.lang.ref.PhantomReference; 21 | import java.lang.ref.ReferenceQueue; 22 | import java.util.Set; 23 | import java.util.concurrent.ConcurrentHashMap; 24 | import java.util.concurrent.atomic.AtomicBoolean; 25 | 26 | import static java.util.Objects.requireNonNull; 27 | 28 | public class FinalizerService 29 | { 30 | public static final Log log = LogFactory.getLog(FinalizerService.class); 31 | 32 | private static final long REMOVE_TIMEOUT = 5000; 33 | 34 | private static FinalizerService instance; 35 | 36 | private final Set finalizers = Sets.newSetFromMap(new ConcurrentHashMap()); 37 | private final ReferenceQueue finalizerQueue = new ReferenceQueue<>(); 38 | private Thread finalizerThread; 39 | 40 | private FinalizerService() {} 41 | 42 | public static synchronized FinalizerService getInstance() 43 | { 44 | if (instance == null) { 45 | FinalizerService finalizer = new FinalizerService(); 46 | finalizer.start(); 47 | instance = finalizer; 48 | } 49 | return instance; 50 | } 51 | 52 | private void start() 53 | { 54 | if (finalizerThread != null) { 55 | return; 56 | } 57 | finalizerThread = new Thread(new Runnable() 58 | { 59 | @Override 60 | public void run() 61 | { 62 | processFinalizerQueue(); 63 | } 64 | }); 65 | finalizerThread.setDaemon(true); 66 | finalizerThread.setName("hadoop-finalizer-service"); 67 | finalizerThread.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() 68 | { 69 | @Override 70 | public void uncaughtException(Thread t, Throwable e) 71 | { 72 | log.error("Uncaught exception in finalizer thread", e); 73 | } 74 | }); 75 | finalizerThread.start(); 76 | } 77 | 78 | /** 79 | * When referent is freed by the garbage collector, run cleanup. 80 | *

81 | * Note: cleanup must not contain a reference to the referent object. 82 | */ 83 | public void addFinalizer(Object referent, Runnable cleanup) 84 | { 85 | requireNonNull(referent, "referent is null"); 86 | requireNonNull(cleanup, "cleanup is null"); 87 | finalizers.add(new FinalizerReference(referent, finalizerQueue, cleanup)); 88 | } 89 | 90 | private void processFinalizerQueue() 91 | { 92 | while (!Thread.currentThread().isInterrupted()) { 93 | try { 94 | FinalizerReference finalizer = (FinalizerReference) finalizerQueue.remove(REMOVE_TIMEOUT); 95 | if (finalizer != null) { 96 | finalizers.remove(finalizer); 97 | finalizer.cleanup(); 98 | } 99 | } 100 | catch (InterruptedException e) { 101 | return; 102 | } 103 | catch (Throwable e) { 104 | log.error("Exception in finalizer queue processor", e); 105 | } 106 | } 107 | } 108 | 109 | private static class FinalizerReference 110 | extends PhantomReference 111 | { 112 | private final Runnable cleanup; 113 | private final AtomicBoolean executed = new AtomicBoolean(); 114 | 115 | public FinalizerReference(Object referent, ReferenceQueue queue, Runnable cleanup) 116 | { 117 | super(requireNonNull(referent, "referent is null"), requireNonNull(queue, "queue is null")); 118 | this.cleanup = requireNonNull(cleanup, "cleanup is null"); 119 | } 120 | 121 | public void cleanup() 122 | { 123 | if (executed.compareAndSet(false, true)) { 124 | cleanup.run(); 125 | } 126 | } 127 | } 128 | } 129 | -------------------------------------------------------------------------------- /src/main/java/org/apache/hadoop/fs/InputStreamWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package org.apache.hadoop.fs; 15 | 16 | public class InputStreamWrapper 17 | extends FSDataInputStream 18 | { 19 | private final FileSystem fileSystem; 20 | 21 | public InputStreamWrapper(FSDataInputStream inputStream, FileSystem fileSystem) 22 | { 23 | super(inputStream); 24 | this.fileSystem = fileSystem; 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/org/apache/hadoop/fs/OutputStreamWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package org.apache.hadoop.fs; 15 | 16 | import java.io.IOException; 17 | 18 | public class OutputStreamWrapper 19 | extends FSDataOutputStream 20 | { 21 | private final FileSystem fileSystem; 22 | 23 | public OutputStreamWrapper(FSDataOutputStream delegate, FileSystem fileSystem) 24 | throws IOException 25 | { 26 | super(delegate, null, delegate.getPos()); 27 | this.fileSystem = fileSystem; 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /src/main/java/org/apache/hadoop/fs/PrestoFileSystemCache.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package org.apache.hadoop.fs; 15 | 16 | import com.facebook.presto.hadoop.FileSystemFactory; 17 | import com.google.common.collect.ImmutableSet; 18 | import org.apache.commons.logging.Log; 19 | import org.apache.commons.logging.LogFactory; 20 | import org.apache.hadoop.conf.Configuration; 21 | import org.apache.hadoop.security.UserGroupInformation; 22 | import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; 23 | import org.apache.hadoop.util.ReflectionUtils; 24 | 25 | import java.io.IOException; 26 | import java.net.URI; 27 | import java.util.HashMap; 28 | import java.util.Iterator; 29 | import java.util.Map; 30 | import java.util.Map.Entry; 31 | import java.util.Objects; 32 | import java.util.Set; 33 | import java.util.concurrent.atomic.AtomicLong; 34 | 35 | import static com.google.common.base.MoreObjects.toStringHelper; 36 | import static com.google.common.base.Strings.nullToEmpty; 37 | import static java.lang.String.format; 38 | import static java.util.Locale.ENGLISH; 39 | import static java.util.Objects.requireNonNull; 40 | import static org.apache.hadoop.fs.FileSystem.getFileSystemClass; 41 | import static org.apache.hadoop.security.UserGroupInformationShim.getSubject; 42 | 43 | public class PrestoFileSystemCache 44 | extends FileSystem.Cache 45 | { 46 | public static final Log log = LogFactory.getLog(PrestoFileSystemCache.class); 47 | public static final String PRESTO_GCS_OAUTH_ACCESS_TOKEN_KEY = "presto.gcs.oauth-access-token"; 48 | public static final String PRESTO_S3_IAM_ROLE = "presto.hive.s3.iam-role"; 49 | public static final String PRESTO_S3_ACCESS_KEY = "presto.s3.access-key"; 50 | 51 | private final AtomicLong unique = new AtomicLong(); 52 | private final Map map = new HashMap<>(); 53 | 54 | @Override 55 | FileSystem get(URI uri, Configuration conf) 56 | throws IOException 57 | { 58 | if (conf instanceof FileSystemFactory) { 59 | return ((FileSystemFactory) conf).createFileSystem(uri); 60 | } 61 | 62 | return getInternal(uri, conf, 0); 63 | } 64 | 65 | @Override 66 | FileSystem getUnique(URI uri, Configuration conf) 67 | throws IOException 68 | { 69 | if (conf instanceof FileSystemFactory) { 70 | return ((FileSystemFactory) conf).createFileSystem(uri); 71 | } 72 | 73 | return getInternal(uri, conf, unique.incrementAndGet()); 74 | } 75 | 76 | private synchronized FileSystem getInternal(URI uri, Configuration conf, long unique) 77 | throws IOException 78 | { 79 | UserGroupInformation userGroupInformation = UserGroupInformation.getCurrentUser(); 80 | FileSystemKey key = createFileSystemKey(uri, userGroupInformation, unique); 81 | Set privateCredentials = getPrivateCredentials(userGroupInformation); 82 | 83 | FileSystemHolder fileSystemHolder = map.get(key); 84 | if (fileSystemHolder == null) { 85 | int maxSize = conf.getInt("fs.cache.max-size", 1000); 86 | if (map.size() >= maxSize) { 87 | throw new IOException(format("FileSystem max cache size has been reached: %s", maxSize)); 88 | } 89 | FileSystem fileSystem = createFileSystem(uri, conf); 90 | fileSystemHolder = new FileSystemHolder(fileSystem, privateCredentials); 91 | map.put(key, fileSystemHolder); 92 | } 93 | 94 | // Private credentials are only set when using Kerberos authentication. 95 | // When the user is the same, but the private credentials are different, 96 | // that means that Kerberos ticket has expired and re-login happened. 97 | // To prevent cache leak in such situation, the privateCredentials are not 98 | // a part of the FileSystemKey, but part of the FileSystemHolder. When a 99 | // Kerberos re-login occurs, re-create the file system and cache it using 100 | // the same key. 101 | if (fileSystemRefresh(uri, conf, privateCredentials, fileSystemHolder)) { 102 | map.remove(key); 103 | FileSystem fileSystem = createFileSystem(uri, conf); 104 | fileSystemHolder = new FileSystemHolder(fileSystem, privateCredentials); 105 | map.put(key, fileSystemHolder); 106 | } 107 | 108 | return fileSystemHolder.getFileSystem(); 109 | } 110 | 111 | private boolean fileSystemRefresh(URI uri, Configuration conf, Set privateCredentials, FileSystemHolder fileSystemHolder) 112 | { 113 | if (isHdfs(uri)) { 114 | return !fileSystemHolder.getPrivateCredentials().equals(privateCredentials); 115 | } 116 | if ("gs".equals(uri.getScheme())) { 117 | String existingGcsToken = fileSystemHolder.getFileSystem().getConf().get(PRESTO_GCS_OAUTH_ACCESS_TOKEN_KEY); 118 | String newGcsToken = conf.get(PRESTO_GCS_OAUTH_ACCESS_TOKEN_KEY); 119 | if (existingGcsToken == null) { 120 | return newGcsToken != null; 121 | } 122 | return !existingGcsToken.equals(newGcsToken); 123 | } 124 | if (uri.getScheme().startsWith("s3")) { 125 | String existingIAMRole = fileSystemHolder.getFileSystem().getConf().get(PRESTO_S3_IAM_ROLE); 126 | String existingAccessKey = fileSystemHolder.getFileSystem().getConf().get(PRESTO_S3_ACCESS_KEY); 127 | String newIAMRole = conf.get(PRESTO_S3_IAM_ROLE); 128 | String newAccessKey = conf.get(PRESTO_S3_ACCESS_KEY); 129 | 130 | if (existingAccessKey == null && existingIAMRole == null) { 131 | return newIAMRole != null || newAccessKey != null; 132 | } 133 | if (existingIAMRole != null) { 134 | return !existingIAMRole.equals(newIAMRole); 135 | } 136 | return !existingAccessKey.equals(newAccessKey); 137 | } 138 | return false; 139 | } 140 | 141 | private FileSystem createFileSystem(URI uri, Configuration conf) 142 | throws IOException 143 | { 144 | Class clazz = getFileSystemClass(uri.getScheme(), conf); 145 | if (clazz == null) { 146 | throw new IOException("No FileSystem for scheme: " + uri.getScheme()); 147 | } 148 | final FileSystem original = (FileSystem) ReflectionUtils.newInstance(clazz, conf); 149 | original.initialize(uri, conf); 150 | FileSystem wrapper = createPrestoFileSystemWrapper(original); 151 | FinalizerService.getInstance().addFinalizer(wrapper, new Runnable() 152 | { 153 | @Override 154 | public void run() 155 | { 156 | try { 157 | original.close(); 158 | } 159 | catch (IOException e) { 160 | log.error("Error occurred when finalizing file system", e); 161 | } 162 | } 163 | }); 164 | return wrapper; 165 | } 166 | 167 | protected FileSystem createPrestoFileSystemWrapper(FileSystem original) 168 | { 169 | return new PrestoFilterFileSystemWrapper(original); 170 | } 171 | 172 | @Override 173 | synchronized void remove(Key ignored, FileSystem fileSystem) 174 | { 175 | if (fileSystem == null) { 176 | return; 177 | } 178 | Iterator> iterator = map.entrySet().iterator(); 179 | while (iterator.hasNext()) { 180 | if (fileSystem.equals(iterator.next().getValue().getFileSystem())) { 181 | iterator.remove(); 182 | break; 183 | } 184 | } 185 | } 186 | 187 | @Override 188 | synchronized void closeAll() 189 | throws IOException 190 | { 191 | for (FileSystemHolder fileSystemHolder : map.values()) { 192 | fileSystemHolder.getFileSystem().close(); 193 | } 194 | map.clear(); 195 | } 196 | 197 | @Override 198 | synchronized void closeAll(boolean onlyAutomatic) 199 | throws IOException 200 | { 201 | throw new UnsupportedOperationException(); 202 | } 203 | 204 | @Override 205 | synchronized void closeAll(UserGroupInformation ugi) 206 | throws IOException 207 | { 208 | throw new UnsupportedOperationException(); 209 | } 210 | 211 | private static FileSystemKey createFileSystemKey(URI uri, UserGroupInformation userGroupInformation, long unique) 212 | { 213 | String scheme = nullToEmpty(uri.getScheme()).toLowerCase(ENGLISH); 214 | String authority = nullToEmpty(uri.getAuthority()).toLowerCase(ENGLISH); 215 | String realUser; 216 | String proxyUser; 217 | AuthenticationMethod authenticationMethod = userGroupInformation.getAuthenticationMethod(); 218 | switch (authenticationMethod) { 219 | case SIMPLE: 220 | case KERBEROS: 221 | realUser = userGroupInformation.getUserName(); 222 | proxyUser = null; 223 | break; 224 | case PROXY: 225 | realUser = userGroupInformation.getRealUser().getUserName(); 226 | proxyUser = userGroupInformation.getUserName(); 227 | break; 228 | default: 229 | throw new IllegalArgumentException("Unsupported authentication method: " + authenticationMethod); 230 | } 231 | return new FileSystemKey(scheme, authority, unique, realUser, proxyUser); 232 | } 233 | 234 | private static Set getPrivateCredentials(UserGroupInformation userGroupInformation) 235 | { 236 | AuthenticationMethod authenticationMethod = userGroupInformation.getAuthenticationMethod(); 237 | switch (authenticationMethod) { 238 | case SIMPLE: 239 | return ImmutableSet.of(); 240 | case KERBEROS: 241 | return ImmutableSet.copyOf(getSubject(userGroupInformation).getPrivateCredentials()); 242 | case PROXY: 243 | return getPrivateCredentials(userGroupInformation.getRealUser()); 244 | default: 245 | throw new IllegalArgumentException("Unsupported authentication method: " + authenticationMethod); 246 | } 247 | } 248 | 249 | private static boolean isHdfs(URI uri) 250 | { 251 | String scheme = uri.getScheme(); 252 | return "hdfs".equals(scheme) || "viewfs".equals(scheme); 253 | } 254 | 255 | private static class FileSystemKey 256 | { 257 | private final String scheme; 258 | private final String authority; 259 | private final long unique; 260 | private final String realUser; 261 | private final String proxyUser; 262 | 263 | public FileSystemKey(String scheme, String authority, long unique, String realUser, String proxyUser) 264 | { 265 | this.scheme = requireNonNull(scheme, "scheme is null"); 266 | this.authority = requireNonNull(authority, "authority is null"); 267 | this.unique = unique; 268 | this.realUser = requireNonNull(realUser, "realUser"); 269 | this.proxyUser = proxyUser; 270 | } 271 | 272 | @Override 273 | public boolean equals(Object o) 274 | { 275 | if (this == o) { 276 | return true; 277 | } 278 | if (o == null || getClass() != o.getClass()) { 279 | return false; 280 | } 281 | FileSystemKey that = (FileSystemKey) o; 282 | return Objects.equals(scheme, that.scheme) && 283 | Objects.equals(authority, that.authority) && 284 | Objects.equals(unique, that.unique) && 285 | Objects.equals(realUser, that.realUser) && 286 | Objects.equals(proxyUser, that.proxyUser); 287 | } 288 | 289 | @Override 290 | public int hashCode() 291 | { 292 | return Objects.hash(scheme, authority, unique, realUser, proxyUser); 293 | } 294 | 295 | @Override 296 | public String toString() 297 | { 298 | return toStringHelper(this) 299 | .add("scheme", scheme) 300 | .add("authority", authority) 301 | .add("unique", unique) 302 | .add("realUser", realUser) 303 | .add("proxyUser", proxyUser) 304 | .toString(); 305 | } 306 | } 307 | 308 | private static class FileSystemHolder 309 | { 310 | private final FileSystem fileSystem; 311 | private final Set privateCredentials; 312 | 313 | public FileSystemHolder(FileSystem fileSystem, Set privateCredentials) 314 | { 315 | this.fileSystem = requireNonNull(fileSystem, "fileSystem is null"); 316 | this.privateCredentials = ImmutableSet.copyOf(requireNonNull(privateCredentials, "privateCredentials is null")); 317 | } 318 | 319 | public FileSystem getFileSystem() 320 | { 321 | return fileSystem; 322 | } 323 | 324 | public Set getPrivateCredentials() 325 | { 326 | return privateCredentials; 327 | } 328 | 329 | @Override 330 | public String toString() 331 | { 332 | return toStringHelper(this) 333 | .add("fileSystem", fileSystem) 334 | .add("privateCredentials", privateCredentials) 335 | .toString(); 336 | } 337 | } 338 | } 339 | -------------------------------------------------------------------------------- /src/main/java/org/apache/hadoop/fs/PrestoFilterFileSystemWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package org.apache.hadoop.fs; 15 | 16 | import org.apache.hadoop.fs.permission.FsPermission; 17 | import org.apache.hadoop.util.Progressable; 18 | 19 | import java.io.IOException; 20 | import java.util.EnumSet; 21 | 22 | public class PrestoFilterFileSystemWrapper 23 | extends FilterFileSystem 24 | { 25 | public PrestoFilterFileSystemWrapper(FileSystem fs) 26 | { 27 | super(fs); 28 | } 29 | 30 | @Override 31 | public FSDataInputStream open(Path f, int bufferSize) 32 | throws IOException 33 | { 34 | return new InputStreamWrapper(getRawFileSystem().open(f, bufferSize), this); 35 | } 36 | 37 | @Override 38 | public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) 39 | throws IOException 40 | { 41 | return new OutputStreamWrapper(getRawFileSystem().append(f, bufferSize, progress), this); 42 | } 43 | 44 | @Override 45 | public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) 46 | throws IOException 47 | { 48 | return new OutputStreamWrapper(getRawFileSystem().create(f, permission, overwrite, bufferSize, replication, blockSize, progress), this); 49 | } 50 | 51 | @Override 52 | public FSDataOutputStream create(Path f, FsPermission permission, EnumSet flags, int bufferSize, short replication, long blockSize, Progressable progress, Options.ChecksumOpt checksumOpt) 53 | throws IOException 54 | { 55 | return new OutputStreamWrapper(getRawFileSystem().create(f, permission, flags, bufferSize, replication, blockSize, progress, checksumOpt), this); 56 | } 57 | 58 | @Override 59 | public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, EnumSet flags, int bufferSize, short replication, long blockSize, Progressable progress) 60 | throws IOException 61 | { 62 | return new OutputStreamWrapper(getRawFileSystem().createNonRecursive(f, permission, flags, bufferSize, replication, blockSize, progress), this); 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /src/main/java/org/apache/hadoop/security/UserGroupInformationShim.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package org.apache.hadoop.security; 15 | 16 | import javax.security.auth.Subject; 17 | import javax.security.auth.kerberos.KerberosPrincipal; 18 | 19 | import java.util.Set; 20 | 21 | import static java.util.Objects.requireNonNull; 22 | import static org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.KERBEROS; 23 | 24 | public final class UserGroupInformationShim 25 | { 26 | private UserGroupInformationShim() {} 27 | 28 | public static Subject getSubject(UserGroupInformation userGroupInformation) 29 | { 30 | return userGroupInformation.getSubject(); 31 | } 32 | 33 | public static UserGroupInformation createUserGroupInformationForSubject(Subject subject) 34 | { 35 | requireNonNull(subject, "subject is null"); 36 | Set kerberosPrincipals = subject.getPrincipals(KerberosPrincipal.class); 37 | if (kerberosPrincipals.isEmpty()) { 38 | throw new IllegalArgumentException("subject must contain a KerberosPrincipal"); 39 | } 40 | if (kerberosPrincipals.size() != 1) { 41 | throw new IllegalArgumentException("subject must contain only a single KerberosPrincipal"); 42 | } 43 | 44 | KerberosPrincipal principal = kerberosPrincipals.iterator().next(); 45 | User user = new User(principal.getName(), KERBEROS, null); 46 | subject.getPrincipals().add(user); 47 | UserGroupInformation userGroupInformation = new UserGroupInformation(subject); 48 | userGroupInformation.setAuthenticationMethod(KERBEROS); 49 | return userGroupInformation; 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /src/main/java/org/apache/hadoop/util/LineReader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package org.apache.hadoop.util; 15 | 16 | import com.facebook.presto.hadoop.TextLineLengthLimitExceededException; 17 | import org.apache.hadoop.classification.InterfaceAudience; 18 | import org.apache.hadoop.classification.InterfaceStability; 19 | import org.apache.hadoop.conf.Configuration; 20 | import org.apache.hadoop.io.Text; 21 | 22 | import java.io.Closeable; 23 | import java.io.IOException; 24 | import java.io.InputStream; 25 | 26 | /** 27 | * A class that provides a line reader from an input stream. 28 | * Depending on the constructor used, lines will either be terminated by: 29 | *
    30 | *
  • one of the following: '\n' (LF) , '\r' (CR), 31 | * or '\r\n' (CR+LF).
  • 32 | *
  • or, a custom byte sequence delimiter
  • 33 | *
34 | * In both cases, EOF also terminates an otherwise unterminated 35 | * line. 36 | */ 37 | @InterfaceAudience.LimitedPrivate("MapReduce") 38 | @InterfaceStability.Unstable 39 | public class LineReader 40 | implements Closeable 41 | { 42 | // Limitation for array size is VM specific. Current HotSpot VM limitation 43 | // for array size is Integer.MAX_VALUE - 5 (2^31 - 1 - 5). 44 | // Integer.MAX_VALUE - 8 should be safe enough. 45 | private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8; 46 | private static final int DEFAULT_BUFFER_SIZE = 64 * 1024; 47 | private static final byte CR = '\r'; 48 | private static final byte LF = '\n'; 49 | // The line delimiter 50 | private final byte[] recordDelimiterBytes; 51 | private int bufferSize = DEFAULT_BUFFER_SIZE; 52 | private InputStream in; 53 | private byte[] buffer; 54 | // the number of bytes of real data in the buffer 55 | private int bufferLength; 56 | // the current position in the buffer 57 | private int bufferPosn; 58 | 59 | /** 60 | * Create a line reader that reads from the given stream using the 61 | * default buffer-size (64k). 62 | * 63 | * @param in The input stream 64 | * @throws IOException 65 | */ 66 | public LineReader(InputStream in) 67 | { 68 | this(in, DEFAULT_BUFFER_SIZE); 69 | } 70 | 71 | /** 72 | * Create a line reader that reads from the given stream using the 73 | * given buffer-size. 74 | * 75 | * @param in The input stream 76 | * @param bufferSize Size of the read buffer 77 | * @throws IOException 78 | */ 79 | public LineReader(InputStream in, int bufferSize) 80 | { 81 | this.in = in; 82 | this.bufferSize = bufferSize; 83 | this.buffer = new byte[this.bufferSize]; 84 | this.recordDelimiterBytes = null; 85 | } 86 | 87 | /** 88 | * Create a line reader that reads from the given stream using the 89 | * io.file.buffer.size specified in the given 90 | * Configuration. 91 | * 92 | * @param in input stream 93 | * @param conf configuration 94 | * @throws IOException 95 | */ 96 | public LineReader(InputStream in, Configuration conf) 97 | throws IOException 98 | { 99 | this(in, conf.getInt("io.file.buffer.size", DEFAULT_BUFFER_SIZE)); 100 | } 101 | 102 | /** 103 | * Create a line reader that reads from the given stream using the 104 | * default buffer-size, and using a custom delimiter of array of 105 | * bytes. 106 | * 107 | * @param in The input stream 108 | * @param recordDelimiterBytes The delimiter 109 | */ 110 | public LineReader(InputStream in, byte[] recordDelimiterBytes) 111 | { 112 | this.in = in; 113 | this.bufferSize = DEFAULT_BUFFER_SIZE; 114 | this.buffer = new byte[this.bufferSize]; 115 | this.recordDelimiterBytes = recordDelimiterBytes; 116 | } 117 | 118 | /** 119 | * Create a line reader that reads from the given stream using the 120 | * given buffer-size, and using a custom delimiter of array of 121 | * bytes. 122 | * 123 | * @param in The input stream 124 | * @param bufferSize Size of the read buffer 125 | * @param recordDelimiterBytes The delimiter 126 | * @throws IOException 127 | */ 128 | public LineReader(InputStream in, int bufferSize, 129 | byte[] recordDelimiterBytes) 130 | { 131 | this.in = in; 132 | this.bufferSize = bufferSize; 133 | this.buffer = new byte[this.bufferSize]; 134 | this.recordDelimiterBytes = recordDelimiterBytes; 135 | } 136 | 137 | /** 138 | * Create a line reader that reads from the given stream using the 139 | * io.file.buffer.size specified in the given 140 | * Configuration, and using a custom delimiter of array of 141 | * bytes. 142 | * 143 | * @param in input stream 144 | * @param conf configuration 145 | * @param recordDelimiterBytes The delimiter 146 | * @throws IOException 147 | */ 148 | public LineReader(InputStream in, Configuration conf, 149 | byte[] recordDelimiterBytes) 150 | throws IOException 151 | { 152 | this.in = in; 153 | this.bufferSize = conf.getInt("io.file.buffer.size", DEFAULT_BUFFER_SIZE); 154 | this.buffer = new byte[this.bufferSize]; 155 | this.recordDelimiterBytes = recordDelimiterBytes; 156 | } 157 | 158 | /** 159 | * Close the underlying stream. 160 | * 161 | * @throws IOException 162 | */ 163 | public void close() 164 | throws IOException 165 | { 166 | in.close(); 167 | } 168 | 169 | /** 170 | * Read one line from the InputStream into the given Text. 171 | * 172 | * @param str the object to store the given line (without newline) 173 | * @param maxLineLength the maximum number of bytes to store into str; 174 | * the rest of the line is silently discarded. 175 | * @param maxBytesToConsume the maximum number of bytes to consume 176 | * in this call. This is only a hint, because if the line cross 177 | * this threshold, we allow it to happen. It can overshoot 178 | * potentially by as much as one buffer length. 179 | * @return the number of bytes read including the (longest) newline 180 | * found. 181 | * @throws IOException if the underlying stream throws 182 | */ 183 | public int readLine(Text str, int maxLineLength, 184 | int maxBytesToConsume) 185 | throws IOException 186 | { 187 | maxLineLength = Math.min(maxLineLength, MAX_ARRAY_SIZE); 188 | maxBytesToConsume = Math.min(maxBytesToConsume, MAX_ARRAY_SIZE); 189 | if (this.recordDelimiterBytes != null) { 190 | return readCustomLine(str, maxLineLength, maxBytesToConsume); 191 | } 192 | else { 193 | return readDefaultLine(str, maxLineLength, maxBytesToConsume); 194 | } 195 | } 196 | 197 | protected int fillBuffer(InputStream in, byte[] buffer, boolean inDelimiter) 198 | throws IOException 199 | { 200 | return in.read(buffer); 201 | } 202 | 203 | /** 204 | * Read a line terminated by one of CR, LF, or CRLF. 205 | */ 206 | private int readDefaultLine(Text str, int maxLineLength, int maxBytesToConsume) 207 | throws IOException 208 | { 209 | /* We're reading data from in, but the head of the stream may be 210 | * already buffered in buffer, so we have several cases: 211 | * 1. No newline characters are in the buffer, so we need to copy 212 | * everything and read another buffer from the stream. 213 | * 2. An unambiguously terminated line is in buffer, so we just 214 | * copy to str. 215 | * 3. Ambiguously terminated line is in buffer, i.e. buffer ends 216 | * in CR. In this case we copy everything up to CR to str, but 217 | * we also need to see what follows CR: if it's LF, then we 218 | * need consume LF as well, so next call to readLine will read 219 | * from after that. 220 | * We use a flag prevCharCR to signal if previous character was CR 221 | * and, if it happens to be at the end of the buffer, delay 222 | * consuming it until we have a chance to look at the char that 223 | * follows. 224 | */ 225 | str.clear(); 226 | int txtLength = 0; //tracks str.getLength(), as an optimization 227 | int newlineLength = 0; //length of terminating newline 228 | boolean prevCharCR = false; //true of prev char was CR 229 | long bytesConsumed = 0; 230 | do { 231 | int startPosn = bufferPosn; //starting from where we left off the last time 232 | if (bufferPosn >= bufferLength) { 233 | startPosn = 0; 234 | bufferPosn = 0; 235 | if (prevCharCR) { 236 | ++bytesConsumed; //account for CR from previous read 237 | } 238 | bufferLength = fillBuffer(in, buffer, prevCharCR); 239 | if (bufferLength <= 0) { 240 | break; // EOF 241 | } 242 | } 243 | for (; bufferPosn < bufferLength; ++bufferPosn) { //search for newline 244 | if (buffer[bufferPosn] == LF) { 245 | newlineLength = (prevCharCR) ? 2 : 1; 246 | ++bufferPosn; // at next invocation proceed from following byte 247 | break; 248 | } 249 | if (prevCharCR) { //CR + notLF, we are at notLF 250 | newlineLength = 1; 251 | break; 252 | } 253 | prevCharCR = (buffer[bufferPosn] == CR); 254 | } 255 | int readLength = bufferPosn - startPosn; 256 | if (prevCharCR && newlineLength == 0) { 257 | --readLength; //CR at the end of the buffer 258 | } 259 | bytesConsumed += readLength; 260 | int appendLength = readLength - newlineLength; 261 | if (appendLength > maxLineLength - txtLength) { 262 | appendLength = maxLineLength - txtLength; 263 | if (appendLength > 0) { 264 | // We want to fail the read when the line length is over the limit. 265 | throw new TextLineLengthLimitExceededException("Too many bytes before newline: " + maxLineLength); 266 | } 267 | } 268 | if (appendLength > 0) { 269 | int newTxtLength = txtLength + appendLength; 270 | if (str.getBytes().length < newTxtLength && Math.max(newTxtLength, txtLength << 1) > MAX_ARRAY_SIZE) { 271 | // If str need to be resized but the target capacity is over VM limit, it will trigger OOM. 272 | // In such case we will throw an IOException so the caller can deal with it. 273 | throw new TextLineLengthLimitExceededException("Too many bytes before newline: " + newTxtLength); 274 | } 275 | str.append(buffer, startPosn, appendLength); 276 | txtLength = newTxtLength; 277 | } 278 | } 279 | while (newlineLength == 0 && bytesConsumed < maxBytesToConsume); 280 | 281 | if (newlineLength == 0 && bytesConsumed >= maxBytesToConsume) { 282 | // It is possible that bytesConsumed is over the maxBytesToConsume but we 283 | // didn't append anything to str.bytes. If we have consumed over maxBytesToConsume 284 | // bytes but still haven't seen a line terminator, we will fail the read. 285 | throw new TextLineLengthLimitExceededException("Too many bytes before newline: " + bytesConsumed); 286 | } 287 | return (int) bytesConsumed; 288 | } 289 | 290 | /** 291 | * Read a line terminated by a custom delimiter. 292 | */ 293 | private int readCustomLine(Text str, int maxLineLength, int maxBytesToConsume) 294 | throws IOException 295 | { 296 | /* We're reading data from inputStream, but the head of the stream may be 297 | * already captured in the previous buffer, so we have several cases: 298 | * 299 | * 1. The buffer tail does not contain any character sequence which 300 | * matches with the head of delimiter. We count it as a 301 | * ambiguous byte count = 0 302 | * 303 | * 2. The buffer tail contains a X number of characters, 304 | * that forms a sequence, which matches with the 305 | * head of delimiter. We count ambiguous byte count = X 306 | * 307 | * // *** eg: A segment of input file is as follows 308 | * 309 | * " record 1792: I found this bug very interesting and 310 | * I have completely read about it. record 1793: This bug 311 | * can be solved easily record 1794: This ." 312 | * 313 | * delimiter = "record"; 314 | * 315 | * supposing:- String at the end of buffer = 316 | * "I found this bug very interesting and I have completely re" 317 | * There for next buffer = "ad about it. record 179 ...." 318 | * 319 | * The matching characters in the input 320 | * buffer tail and delimiter head = "re" 321 | * Therefore, ambiguous byte count = 2 **** // 322 | * 323 | * 2.1 If the following bytes are the remaining characters of 324 | * the delimiter, then we have to capture only up to the starting 325 | * position of delimiter. That means, we need not include the 326 | * ambiguous characters in str. 327 | * 328 | * 2.2 If the following bytes are not the remaining characters of 329 | * the delimiter ( as mentioned in the example), 330 | * then we have to include the ambiguous characters in str. 331 | */ 332 | str.clear(); 333 | int txtLength = 0; // tracks str.getLength(), as an optimization 334 | long bytesConsumed = 0; 335 | int delPosn = 0; 336 | int ambiguousByteCount = 0; // To capture the ambiguous characters count 337 | do { 338 | int startPosn = bufferPosn; // Start from previous end position 339 | if (bufferPosn >= bufferLength) { 340 | startPosn = 0; 341 | bufferPosn = 0; 342 | bufferLength = fillBuffer(in, buffer, ambiguousByteCount > 0); 343 | if (bufferLength <= 0) { 344 | if (ambiguousByteCount > 0) { 345 | str.append(recordDelimiterBytes, 0, ambiguousByteCount); 346 | bytesConsumed += ambiguousByteCount; 347 | } 348 | break; // EOF 349 | } 350 | } 351 | for (; bufferPosn < bufferLength; ++bufferPosn) { 352 | if (buffer[bufferPosn] == recordDelimiterBytes[delPosn]) { 353 | delPosn++; 354 | if (delPosn >= recordDelimiterBytes.length) { 355 | bufferPosn++; 356 | break; 357 | } 358 | } 359 | else if (delPosn != 0) { 360 | bufferPosn -= delPosn; 361 | if (bufferPosn < -1) { 362 | bufferPosn = -1; 363 | } 364 | delPosn = 0; 365 | } 366 | } 367 | int readLength = bufferPosn - startPosn; 368 | bytesConsumed += readLength; 369 | int appendLength = readLength - delPosn; 370 | if (appendLength > maxLineLength - txtLength) { 371 | appendLength = maxLineLength - txtLength; 372 | if (appendLength > 0) { 373 | // We want to fail the read when the line length is over the limit. 374 | throw new TextLineLengthLimitExceededException("Too many bytes before delimiter: " + maxLineLength); 375 | } 376 | } 377 | bytesConsumed += ambiguousByteCount; 378 | if (appendLength >= 0 && ambiguousByteCount > 0) { 379 | //appending the ambiguous characters (refer case 2.2) 380 | str.append(recordDelimiterBytes, 0, ambiguousByteCount); 381 | ambiguousByteCount = 0; 382 | // since it is now certain that the split did not split a delimiter we 383 | // should not read the next record: clear the flag otherwise duplicate 384 | // records could be generated 385 | unsetNeedAdditionalRecordAfterSplit(); 386 | } 387 | if (appendLength > 0) { 388 | int newTxtLength = txtLength + appendLength; 389 | if (str.getBytes().length < newTxtLength && Math.max(newTxtLength, txtLength << 1) > MAX_ARRAY_SIZE) { 390 | // If str need to be resized but the target capacity is over VM limit, it will trigger OOM. 391 | // In such case we will throw an IOException so the caller can deal with it. 392 | throw new TextLineLengthLimitExceededException("Too many bytes before delimiter: " + newTxtLength); 393 | } 394 | str.append(buffer, startPosn, appendLength); 395 | txtLength = newTxtLength; 396 | } 397 | if (bufferPosn >= bufferLength) { 398 | if (delPosn > 0 && delPosn < recordDelimiterBytes.length) { 399 | ambiguousByteCount = delPosn; 400 | bytesConsumed -= ambiguousByteCount; //to be consumed in next 401 | } 402 | } 403 | } 404 | while (delPosn < recordDelimiterBytes.length 405 | && bytesConsumed < maxBytesToConsume); 406 | if (delPosn < recordDelimiterBytes.length 407 | && bytesConsumed >= maxBytesToConsume) { 408 | // It is possible that bytesConsumed is over the maxBytesToConsume but we 409 | // didn't append anything to str.bytes. If we have consumed over maxBytesToConsume 410 | // bytes but still haven't seen a line terminator, we will fail the read. 411 | throw new TextLineLengthLimitExceededException("Too many bytes before delimiter: " + bytesConsumed); 412 | } 413 | return (int) bytesConsumed; 414 | } 415 | 416 | /** 417 | * Read from the InputStream into the given Text. 418 | * 419 | * @param str the object to store the given line 420 | * @param maxLineLength the maximum number of bytes to store into str. 421 | * @return the number of bytes read including the newline 422 | * @throws IOException if the underlying stream throws 423 | */ 424 | public int readLine(Text str, int maxLineLength) 425 | throws IOException 426 | { 427 | return readLine(str, maxLineLength, Integer.MAX_VALUE); 428 | } 429 | 430 | /** 431 | * Read from the InputStream into the given Text. 432 | * 433 | * @param str the object to store the given line 434 | * @return the number of bytes read including the newline 435 | * @throws IOException if the underlying stream throws 436 | */ 437 | public int readLine(Text str) 438 | throws IOException 439 | { 440 | return readLine(str, Integer.MAX_VALUE, Integer.MAX_VALUE); 441 | } 442 | 443 | protected int getBufferPosn() 444 | { 445 | return bufferPosn; 446 | } 447 | 448 | protected int getBufferSize() 449 | { 450 | return bufferSize; 451 | } 452 | 453 | protected void unsetNeedAdditionalRecordAfterSplit() 454 | { 455 | // needed for custom multi byte line delimiters only 456 | // see MAPREDUCE-6549 for details 457 | } 458 | } 459 | -------------------------------------------------------------------------------- /src/main/resources/core-site.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | dfs.client.use.legacy.blockreader 5 | true 6 | 7 | 8 | -------------------------------------------------------------------------------- /src/main/resources/nativelib/Linux-aarch64/libhadoop.so: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/prestodb/presto-hadoop-apache2/484233009784a76d6fc09ae06dad7a51129bf3ce/src/main/resources/nativelib/Linux-aarch64/libhadoop.so -------------------------------------------------------------------------------- /src/main/resources/nativelib/Linux-aarch64/libsnappy.so: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/prestodb/presto-hadoop-apache2/484233009784a76d6fc09ae06dad7a51129bf3ce/src/main/resources/nativelib/Linux-aarch64/libsnappy.so -------------------------------------------------------------------------------- /src/main/resources/nativelib/Linux-amd64/libhadoop.so: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/prestodb/presto-hadoop-apache2/484233009784a76d6fc09ae06dad7a51129bf3ce/src/main/resources/nativelib/Linux-amd64/libhadoop.so -------------------------------------------------------------------------------- /src/main/resources/nativelib/Linux-amd64/libsnappy.so: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/prestodb/presto-hadoop-apache2/484233009784a76d6fc09ae06dad7a51129bf3ce/src/main/resources/nativelib/Linux-amd64/libsnappy.so -------------------------------------------------------------------------------- /src/main/resources/nativelib/Linux-ppc64le/libhadoop.so: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/prestodb/presto-hadoop-apache2/484233009784a76d6fc09ae06dad7a51129bf3ce/src/main/resources/nativelib/Linux-ppc64le/libhadoop.so -------------------------------------------------------------------------------- /src/main/resources/nativelib/Linux-ppc64le/libsnappy.so: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/prestodb/presto-hadoop-apache2/484233009784a76d6fc09ae06dad7a51129bf3ce/src/main/resources/nativelib/Linux-ppc64le/libsnappy.so -------------------------------------------------------------------------------- /src/main/resources/nativelib/Mac_OS_X-aarch64/libhadoop.dylib: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/prestodb/presto-hadoop-apache2/484233009784a76d6fc09ae06dad7a51129bf3ce/src/main/resources/nativelib/Mac_OS_X-aarch64/libhadoop.dylib -------------------------------------------------------------------------------- /src/main/resources/nativelib/Mac_OS_X-aarch64/libsnappy.dylib: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/prestodb/presto-hadoop-apache2/484233009784a76d6fc09ae06dad7a51129bf3ce/src/main/resources/nativelib/Mac_OS_X-aarch64/libsnappy.dylib -------------------------------------------------------------------------------- /src/main/resources/nativelib/Mac_OS_X-x86_64/libhadoop.dylib: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/prestodb/presto-hadoop-apache2/484233009784a76d6fc09ae06dad7a51129bf3ce/src/main/resources/nativelib/Mac_OS_X-x86_64/libhadoop.dylib -------------------------------------------------------------------------------- /src/main/resources/nativelib/Mac_OS_X-x86_64/libsnappy.dylib: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/prestodb/presto-hadoop-apache2/484233009784a76d6fc09ae06dad7a51129bf3ce/src/main/resources/nativelib/Mac_OS_X-x86_64/libsnappy.dylib -------------------------------------------------------------------------------- /src/test/java/com/facebook/presto/hadoop/TestHadoopFileSystemCache.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package com.facebook.presto.hadoop; 15 | 16 | import org.apache.hadoop.conf.Configuration; 17 | import org.apache.hadoop.fs.FileSystem; 18 | import org.testng.annotations.Test; 19 | 20 | import java.net.URI; 21 | 22 | import static org.testng.Assert.assertNotSame; 23 | import static org.testng.Assert.assertSame; 24 | 25 | public class TestHadoopFileSystemCache 26 | { 27 | @Test 28 | public void testCache() 29 | throws Exception 30 | { 31 | HadoopFileSystemCache.initialize(); 32 | 33 | FileSystem.closeAll(); 34 | 35 | Configuration conf = new Configuration(); 36 | URI uri = URI.create("file:///"); 37 | 38 | FileSystem fs1 = FileSystem.get(uri, conf); 39 | FileSystem fs2 = FileSystem.get(uri, conf); 40 | assertSame(fs2, fs1); 41 | 42 | FileSystem.closeAll(); 43 | 44 | FileSystem fs3 = FileSystem.get(uri, conf); 45 | assertNotSame(fs3, fs1); 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /src/test/java/com/facebook/presto/hadoop/TestHadoopNative.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package com.facebook.presto.hadoop; 15 | 16 | import org.apache.hadoop.conf.Configuration; 17 | import org.apache.hadoop.io.compress.CompressionCodec; 18 | import org.apache.hadoop.io.compress.CompressionCodecFactory; 19 | import org.apache.hadoop.io.compress.bzip2.Bzip2Factory; 20 | import org.apache.hadoop.io.compress.zlib.ZlibFactory; 21 | import org.apache.hadoop.util.NativeCodeLoader; 22 | import org.testng.annotations.Test; 23 | 24 | import java.io.ByteArrayInputStream; 25 | import java.io.ByteArrayOutputStream; 26 | import java.io.IOException; 27 | import java.io.InputStream; 28 | import java.io.OutputStream; 29 | 30 | import static java.nio.charset.StandardCharsets.UTF_8; 31 | import static org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses; 32 | import static org.testng.Assert.assertEquals; 33 | import static org.testng.Assert.assertNotNull; 34 | import static org.testng.Assert.assertTrue; 35 | 36 | public class TestHadoopNative 37 | { 38 | @Test 39 | public void testNative() 40 | { 41 | HadoopNative.requireHadoopNative(); 42 | 43 | assertTrue(NativeCodeLoader.isNativeCodeLoaded()); 44 | assertTrue(NativeCodeLoader.buildSupportsSnappy()); 45 | assertTrue(ZlibFactory.isNativeZlibLoaded(new Configuration())); 46 | assertTrue(Bzip2Factory.isNativeBzip2Loaded(new Configuration())); 47 | } 48 | 49 | @Test 50 | public void testCodecRoundTrip() 51 | throws Exception 52 | { 53 | HadoopNative.requireHadoopNative(); 54 | 55 | Configuration conf = new Configuration(); 56 | CompressionCodecFactory factory = new CompressionCodecFactory(conf); 57 | for (Class clazz : getCodecClasses(conf)) { 58 | CompressionCodec codec = factory.getCodecByClassName(clazz.getName()); 59 | assertNotNull(codec, clazz.getName()); 60 | 61 | byte[] expected = "Hello world! Goodbye world!".getBytes(UTF_8); 62 | byte[] actual = decompress(codec, compress(codec, expected)); 63 | assertEquals(actual, expected); 64 | } 65 | } 66 | 67 | private static byte[] compress(CompressionCodec codec, byte[] input) 68 | throws IOException 69 | { 70 | ByteArrayOutputStream bytes = new ByteArrayOutputStream(); 71 | try (OutputStream out = codec.createOutputStream(bytes)) { 72 | out.write(input); 73 | out.close(); 74 | } 75 | return bytes.toByteArray(); 76 | } 77 | 78 | private static byte[] decompress(CompressionCodec codec, byte[] input) 79 | throws IOException 80 | { 81 | ByteArrayOutputStream bytes = new ByteArrayOutputStream(); 82 | try (InputStream in = codec.createInputStream(new ByteArrayInputStream(input))) { 83 | int b; 84 | while ((b = in.read()) != -1) { 85 | bytes.write(b); 86 | } 87 | } 88 | return bytes.toByteArray(); 89 | } 90 | } 91 | -------------------------------------------------------------------------------- /src/test/java/com/facebook/presto/hadoop/TestLineReader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package com.facebook.presto.hadoop; 15 | 16 | import org.apache.hadoop.io.Text; 17 | import org.apache.hadoop.util.LineReader; 18 | import org.testng.annotations.Test; 19 | 20 | import java.io.ByteArrayInputStream; 21 | import java.io.IOException; 22 | import java.io.InputStream; 23 | 24 | import static java.nio.charset.StandardCharsets.UTF_8; 25 | import static org.testng.Assert.assertEquals; 26 | import static org.testng.Assert.fail; 27 | 28 | public class TestLineReader 29 | { 30 | @Test 31 | public void testDefaultReaderZeroBytes() 32 | throws IOException 33 | { 34 | byte[] input = "Hello world! Goodbye world!\n".getBytes(UTF_8); 35 | InputStream in = new ByteArrayInputStream(input); 36 | // Set the LineReader internal read buffer size 4 bytes 37 | LineReader reader = new LineReader(in, 4); 38 | Text str = new Text(); 39 | reader.readLine(str, 0, 30); 40 | // There should not be any bytes read into str because the maxLineLength is 0 41 | assertEquals(str, new Text()); 42 | } 43 | 44 | @Test 45 | public void testDefaultReaderNonZeroBytes() 46 | throws IOException 47 | { 48 | byte[] input = "Hello world! Goodbye world!\n".getBytes(UTF_8); 49 | InputStream in = new ByteArrayInputStream(input); 50 | // Set the LineReader internal read buffer size 4 bytes 51 | LineReader reader = new LineReader(in, 4); 52 | Text str = new Text(); 53 | reader.readLine(str, 30, 30); 54 | // The LineReader does not store the new line character into str, 55 | // so we need to compare just the first 27 characters. 56 | assertEquals(str, new Text("Hello world! Goodbye world!".getBytes(UTF_8))); 57 | } 58 | 59 | @Test 60 | public void testCustomReaderZeroBytes() 61 | throws IOException 62 | { 63 | byte[] input = "Hello world! Goodbye world!\n".getBytes(UTF_8); 64 | byte[] delimiter = "!".getBytes(UTF_8); 65 | InputStream in = new ByteArrayInputStream(input); 66 | // Set the LineReader internal read buffer size 4 bytes 67 | LineReader reader = new LineReader(in, 4, delimiter); 68 | Text str = new Text(); 69 | reader.readLine(str, 0, 30); 70 | // There should not be any bytes read into str because the maxLineLength is 0 71 | assertEquals(str, new Text()); 72 | } 73 | 74 | @Test 75 | public void testCustomReaderNonZeroBytes() 76 | throws IOException 77 | { 78 | byte[] input = "Hello world! Goodbye world!\n".getBytes(UTF_8); 79 | byte[] delimiter = "!".getBytes(UTF_8); 80 | InputStream in = new ByteArrayInputStream(input); 81 | // Set the LineReader internal read buffer size 4 bytes 82 | LineReader reader = new LineReader(in, 4, delimiter); 83 | Text str = new Text(); 84 | reader.readLine(str, 30, 30); 85 | // The first 11 bytes of input should be read into str because the 12th character is the delimiter 86 | assertEquals(str, new Text("Hello world".getBytes(UTF_8))); 87 | } 88 | 89 | @Test 90 | public void testDefaultReaderMaxBytesConsumed() 91 | throws IOException 92 | { 93 | byte[] input = "Hello world! Goodbye world!\n".getBytes(UTF_8); 94 | InputStream in = new ByteArrayInputStream(input); 95 | // Set the LineReader internal read buffer size 4 bytes 96 | LineReader reader = new LineReader(in, 4); 97 | Text str = new Text(); 98 | try { 99 | // The read should fail because the line length is greater than the maxBytesToConsume 100 | reader.readLine(str, 0, 10); 101 | fail("Expected exception"); 102 | } 103 | catch (TextLineLengthLimitExceededException e) { 104 | // It should be 3 reads of 4 bytes each, so the final bytesConsumed is 12 105 | assertEquals(e.getMessage(), "Too many bytes before newline: 12"); 106 | } 107 | } 108 | 109 | @Test 110 | public void testDefaultReaderMaxLineLength() 111 | throws IOException 112 | { 113 | byte[] input = "Hello world! Goodbye world!\n".getBytes(UTF_8); 114 | InputStream in = new ByteArrayInputStream(input); 115 | // Set the LineReader internal read buffer size 4 bytes 116 | LineReader reader = new LineReader(in, 4); 117 | Text str = new Text(); 118 | try { 119 | // The read should fail because the line length is greater than the maxLineLength 120 | reader.readLine(str, 10, 100); 121 | fail("Expected exception"); 122 | } 123 | catch (TextLineLengthLimitExceededException e) { 124 | assertEquals(e.getMessage(), "Too many bytes before newline: 10"); 125 | } 126 | } 127 | 128 | @Test 129 | public void testCustomReaderMaxBytesConsumed() 130 | throws IOException 131 | { 132 | byte[] input = "Hello world! Goodbye world!\n".getBytes(UTF_8); 133 | byte[] delimiter = "!".getBytes(UTF_8); 134 | InputStream in = new ByteArrayInputStream(input); 135 | // Set the LineReader internal read buffer size 4 bytes 136 | LineReader reader = new LineReader(in, 4, delimiter); 137 | Text str = new Text(); 138 | try { 139 | // The read should fail because the line length is greater than the maxBytesToConsume 140 | reader.readLine(str, 0, 5); 141 | fail("Expected exception"); 142 | } 143 | catch (TextLineLengthLimitExceededException e) { 144 | // It should be 2 reads of 4 bytes each, so the final bytesConsumed is 8 145 | assertEquals(e.getMessage(), "Too many bytes before delimiter: 8"); 146 | } 147 | } 148 | 149 | @Test 150 | public void testCustomReaderMaxLineLength() 151 | throws IOException 152 | { 153 | byte[] input = "Hello world! Goodbye world!\n".getBytes(UTF_8); 154 | byte[] delimiter = "!".getBytes(UTF_8); 155 | InputStream in = new ByteArrayInputStream(input); 156 | // Set the LineReader internal read buffer size 4 bytes 157 | LineReader reader = new LineReader(in, 4, delimiter); 158 | Text str = new Text(); 159 | try { 160 | // The read should fail because the line length is greater than the maxLineLength 161 | reader.readLine(str, 10, 100); 162 | fail("Expected exception"); 163 | } 164 | catch (TextLineLengthLimitExceededException e) { 165 | assertEquals(e.getMessage(), "Too many bytes before delimiter: 10"); 166 | } 167 | } 168 | } 169 | --------------------------------------------------------------------------------