├── .gitignore
├── LICENSE.txt
├── NOTICE.txt
├── README.md
├── bin
└── nnproxy
├── hadoop-nnproxy
├── pom.xml
└── src
│ ├── main
│ ├── java
│ │ └── com
│ │ │ └── bytedance
│ │ │ └── hadoop
│ │ │ └── hdfs
│ │ │ ├── BDManifest.java
│ │ │ ├── ProxyConfig.java
│ │ │ ├── server
│ │ │ ├── NNProxy.java
│ │ │ ├── ProxyMain.java
│ │ │ ├── cache
│ │ │ │ └── CacheRegistry.java
│ │ │ ├── exception
│ │ │ │ └── WrappedExecutionException.java
│ │ │ ├── mount
│ │ │ │ ├── BlockPoolRegistry.java
│ │ │ │ └── MountsManager.java
│ │ │ ├── proxy
│ │ │ │ ├── ProxyClientProtocolHandler.java
│ │ │ │ ├── ProxyMetrics.java
│ │ │ │ ├── ProxyServer.java
│ │ │ │ ├── RouteInfo.java
│ │ │ │ ├── Router.java
│ │ │ │ └── RpcInvocationProxy.java
│ │ │ ├── quota
│ │ │ │ └── ThrottleInvocationHandler.java
│ │ │ └── upstream
│ │ │ │ └── UpstreamManager.java
│ │ │ └── tools
│ │ │ ├── DumpMount.java
│ │ │ └── LoadMount.java
│ └── resources
│ │ └── bdversion.properties
│ └── test
│ └── java
│ ├── com
│ └── bytedance
│ │ └── hadoop
│ │ └── hdfs
│ │ └── server
│ │ ├── MiniNNProxy.java
│ │ └── TestNNProxy.java
│ └── org
│ └── apache
│ └── hadoop
│ └── hdfs
│ └── TestProxyFs.java
└── hadoop-patches
├── 0001-Add-clientAddress-to-rpc-header-For-proxy-usage.patch
└── 0002-Properly-handle-RemoteException.patch
/.gitignore:
--------------------------------------------------------------------------------
1 | .project
2 | .settings
3 | logs
4 | *.log
5 | *.tmp
6 | *.swp
7 | *.classpath
8 | target
9 | install-*
10 | .metadata
11 | *.pyc
12 | .DS_Store
13 | *.iml
14 | .idea
15 | *.class
16 | build
17 | *.hprof
18 | *.dylib
19 | *.so
20 | *.o
21 | *.jar
22 |
--------------------------------------------------------------------------------
/LICENSE.txt:
--------------------------------------------------------------------------------
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 |
179 | APPENDIX: How to apply the Apache License to your work.
180 |
181 | To apply the Apache License to your work, attach the following
182 | boilerplate notice, with the fields enclosed by brackets "[]"
183 | replaced with your own identifying information. (Don't include
184 | the brackets!) The text should be enclosed in the appropriate
185 | comment syntax for the file format. We also recommend that a
186 | file or class name and description of purpose be included on the
187 | same "printed page" as the copyright notice for easier
188 | identification within third-party archives.
189 |
190 | Copyright [yyyy] [name of copyright owner]
191 |
192 | Licensed under the Apache License, Version 2.0 (the "License");
193 | you may not use this file except in compliance with the License.
194 | You may obtain a copy of the License at
195 |
196 | http://www.apache.org/licenses/LICENSE-2.0
197 |
198 | Unless required by applicable law or agreed to in writing, software
199 | distributed under the License is distributed on an "AS IS" BASIS,
200 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
201 | See the License for the specific language governing permissions and
202 | limitations under the License.
203 |
204 |
205 |
--------------------------------------------------------------------------------
/NOTICE.txt:
--------------------------------------------------------------------------------
1 | Copyright 2016 Bytedance Inc.
2 |
3 | Licensed under the Apache License, Version 2.0 (the "License");
4 | you may not use this file except in compliance with the License.
5 | You may obtain a copy of the License at
6 |
7 | http://www.apache.org/licenses/LICENSE-2.0
8 |
9 | Unless required by applicable law or agreed to in writing, software
10 | distributed under the License is distributed on an "AS IS" BASIS,
11 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | See the License for the specific language governing permissions and
13 | limitations under the License.
14 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | # NameNodeProxy
2 |
3 | *Another HDFS Federation solution from [bytedance](http://www.bytedance.com/).*
4 |
5 | ### Features
6 | * NameNode RPC proxy instead of ViewFS, no client-side changes.
7 | * Multiple language client (i.e. snakebite) support.
8 | * Stateless. Easy to scale to multiple instances.
9 | * High throughput with little latency introduced.
10 | * Request throttle. Single NameNode malfunction will not affect globally.
11 |
12 |
13 | Compared with ViewFS and WebHDFS:
14 |
15 | NNProxy ViewFS WebHDFS
16 | Multiple language support Yes No Yes
17 | Unified APIs provided Yes Yes No
18 | Mount table stored in Zookeeper Configuration Configuration
19 | * Client-side configurations are not usually easy to change
20 | Client-side library No Heavy Light
21 | Call latency introduced Medium No latency High
22 | Centralized cache support Yes Yes No
23 |
24 |
25 | ### Benchmark
26 | Benchmark result with NameNode Benchmark 0.4:
27 |
28 | Direct TPS NNProxy TPS (3 proxy instances)
29 | create_write @ 16maps 263,294 247,799
30 | create_write @ 128maps 3,008,604 2,563,465
31 | rename @ 128maps 474,671 471,240
32 |
33 | (numOfFiles 512 each map, NameNode CPU: Xeon E5-2630 v2 * 2, network RTT 0.1ms)
34 |
35 | ### Build
36 |
37 | **IMPORTANT: NNProxy works only with patched NameNode. See `/hadoop-patches` in git repository. These patches add proxy support to `hadoop-common` library. Each of them must be applied before compiling.**
38 |
39 | Build with maven:
40 |
41 | ```
42 | $ cd nnproxy/hadoop-nnproxy
43 | $ mvn package -DskipTests
44 | ```
45 | Built jar are shaded with necessary libraries, which can run with `hadoop jar`.
46 |
47 | ### Deploy
48 |
49 | Organize working directory as follows:
50 | ```
51 | bin/
52 | bin/nnproxy
53 | lib/
54 | lib/hadoop-nnproxy-0.1.0.jar
55 | ```
56 | Then run `bin/nnproxy` with configuration arguments to start proxy server.
57 | Note that before starting proxy server, a mount table must be deployed first.
58 |
59 | For example:
60 | ```
61 | # Setup zk environment, for simplicity, not necessary
62 | export NNPROXY_ZK_QUORUM="127.0.0.1:2181"
63 | export NNPROXY_MOUNT_TABLE_ZKPATH="/hadoop/hdfs/mounts"
64 |
65 | export NNPROXY_OPTS="-Ddfs.nnproxy.mount-table.zk.quorum=$NNPROXY_ZK_QUORUM -Ddfs.nnproxy.mount-table.zk.path=$NNPROXY_MOUNT_TABLE_ZKPATH"
66 |
67 | # Deploy mount table, example above mounts / to hdfs://toutiaonn0 and /data
68 | # to hdfs://toutiaonn1. In which case, accessing /data/recommend would be
69 | # transparently redirected to hdfs://toutiaonn1/data/recommend.
70 | # In this example, toutiaonn0 and toutiaonn1 are fs names configured with
71 | # auto failover HA. Using ip:port directly (non-HA setup) are also fesible.
72 | echo 'hdfs://toutiaonn0 /' > mounts
73 | echo 'hdfs://toutiaonn1 /data' >> mounts
74 | cat mounts | bin/nnproxy load $NNPROXY_OPTS
75 |
76 | # Run proxy (remember these args)
77 | bin/nnproxy proxy $NNPROXY_OPTS
78 | ```
79 | Look into class `com.bytedance.hadoop.hdfs.ProxyConfig` for more configurations available.
80 |
81 | Proxy server metrics are also available with hadoop-metrics2.
82 |
83 | ### Mount table management
84 | Once proxy server is started, it watches mount table node on ZooKeeper for further changes.
85 |
86 | Run `bin/nnproxy dump $NNPROXY_OPTS` to dump current mount table to stdout.
87 |
88 | Run `bin/nnproxy load $NNPROXY_OPTS` to load mount table from stdin.
89 |
90 | One can also alter data stored in mount table ZK node directly (or possibly build APIs or UIs).
91 | All these changes take effect immediately.
92 |
93 | Mount table are delimited text with 3 columns: `targetFS`, `path` and `attributes`, in that order.
94 | For example:
95 | ```
96 | hdfs://toutiaonn0 / none
97 | hdfs://toutiaonn1 /data none
98 | ```
99 | Each row advises proxy to redirect `path` and its children to specified `targetFS`. `attributes` are currently reserved and is of no use.
100 |
101 |
102 | ### Future work
103 | * Support `readonly` attribute in mounts
104 | * More kind of operations through proxy
105 |
106 | ### License
107 | Copyright 2016 Bytedance Inc.
108 |
109 | Licensed under the Apache License, Version 2.0 (the "License");
110 | you may not use this file except in compliance with the License.
111 | You may obtain a copy of the License at
112 |
113 | http://www.apache.org/licenses/LICENSE-2.0
114 |
115 | Unless required by applicable law or agreed to in writing, software
116 | distributed under the License is distributed on an "AS IS" BASIS,
117 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
118 | See the License for the specific language governing permissions and
119 | limitations under the License.
120 |
--------------------------------------------------------------------------------
/bin/nnproxy:
--------------------------------------------------------------------------------
1 | #!/bin/bash
2 | NNPROXY_HOME=`dirname $0`
3 |
4 | DEFAULT_LIBEXEC_DIR=$HADOOP_HOME/libexec
5 | HADOOP_LIBEXEC_DIR=${HADOOP_LIBEXEC_DIR:-$DEFAULT_LIBEXEC_DIR}
6 | . $HADOOP_LIBEXEC_DIR/hdfs-config.sh
7 |
8 | function print_usage(){
9 | echo "Usage: nnproxy COMMAND"
10 | echo " where COMMAND is one of:"
11 | echo " proxy run proxy server"
12 | echo " dump dump mount table (to stdout)"
13 | echo " load load mount table (from stdin)"
14 | echo ""
15 | echo "Most commands print help when invoked w/o parameters."
16 | # There are also debug commands, but they don't show up in this listing.
17 | }
18 |
19 | if [ $# = 0 ]; then
20 | print_usage
21 | exit
22 | fi
23 |
24 | COMMAND=$1
25 | shift
26 |
27 | case $COMMAND in
28 | # usage flags
29 | --help|-help|-h)
30 | print_usage
31 | exit
32 | ;;
33 | esac
34 |
35 | if [ "$COMMAND" = "proxy" ] ; then
36 | CLASS='com.bytedance.hadoop.hdfs.server.ProxyMain'
37 | elif [ "$COMMAND" = "dump" ] ; then
38 | CLASS='com.bytedance.hadoop.hdfs.tools.DumpMount'
39 | elif [ "$COMMAND" = "load" ] ; then
40 | CLASS='com.bytedance.hadoop.hdfs.tools.LoadMount'
41 | fi
42 |
43 | exec $HADOOP_HOME/bin/hadoop jar $NNPROXY_HOME/../lib/hadoop-nnproxy*.jar $CLASS "$@"
44 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/pom.xml:
--------------------------------------------------------------------------------
1 |
4 | 4.0.0
5 |
6 | com.bytedance.hadoop
7 | hadoop-nnproxy
8 | 0.1.0-SNAPSHOT
9 | jar
10 |
11 | hadoop-nnproxy
12 |
13 |
14 | scm:git:https://github.com/bytedance/nnproxy.git
15 |
16 |
17 |
18 | UTF-8
19 | UTF-8
20 | 2.6.0-cdh5.4.4
21 | 3.4.8
22 | 2.9.1
23 | 4.11
24 | 1.10.19
25 | 1.7
26 |
27 |
28 |
29 |
30 | org.apache.hadoop
31 | hadoop-client
32 | ${hadoop.version}
33 |
34 |
35 | commons-configuration
36 | commons-configuration
37 | 1.6
38 |
39 |
40 | org.apache.hadoop
41 | hadoop-common
42 | ${hadoop.version}
43 |
44 |
45 | org.apache.hadoop
46 | hadoop-hdfs
47 | ${hadoop.version}
48 |
49 |
50 | org.apache.curator
51 | curator-framework
52 | ${curator.version}
53 |
54 |
55 | org.apache.curator
56 | curator-client
57 | ${curator.version}
58 |
59 |
60 | org.apache.zookeeper
61 | zookeeper
62 | ${zookeeper.version}
63 |
64 |
65 |
66 | org.apache.hadoop
67 | hadoop-common
68 | ${hadoop.version}
69 | test
70 | test-jar
71 |
72 |
73 | org.apache.hadoop
74 | hadoop-hdfs
75 | ${hadoop.version}
76 | test
77 | test-jar
78 |
79 |
80 | junit
81 | junit
82 | ${junit.version}
83 | test
84 |
85 |
86 | org.mockito
87 | mockito-core
88 | ${mockito.version}
89 | test
90 |
91 |
92 |
93 |
94 |
95 |
96 | src/main/resources
97 | true
98 |
99 |
100 |
101 |
102 | org.codehaus.mojo
103 | buildnumber-maven-plugin
104 | 1.1
105 |
106 |
107 | validate
108 |
109 | create
110 |
111 |
112 |
113 |
114 |
115 | org.apache.maven.plugins
116 | maven-jar-plugin
117 | 2.3.2
118 |
119 |
120 |
121 | ${buildNumber}
122 |
123 |
124 |
125 |
126 |
127 | org.apache.maven.plugins
128 | maven-release-plugin
129 | 2.4.1
130 |
131 |
132 | org.apache.maven.plugins
133 | maven-compiler-plugin
134 | 3.1
135 |
136 | ${java.version}
137 | ${java.version}
138 |
139 |
140 |
141 | org.apache.maven.plugins
142 | maven-shade-plugin
143 | 2.3
144 |
145 |
146 | package
147 |
148 | shade
149 |
150 |
151 | false
152 | false
153 |
154 |
155 | org.apache.curator:*
156 |
157 |
158 |
159 |
160 |
161 |
162 |
163 |
164 |
165 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/BDManifest.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs;
2 |
3 | import org.slf4j.Logger;
4 | import org.slf4j.LoggerFactory;
5 |
6 | import java.io.IOException;
7 | import java.io.InputStream;
8 | import java.util.Properties;
9 |
10 | /** */
11 | public class BDManifest {
12 |
13 | private static final Logger LOG = LoggerFactory.getLogger(BDManifest.class);
14 |
15 | static final Properties properties;
16 |
17 | static {
18 | InputStream inputStream = BDManifest.class.getResourceAsStream("/bdversion.properties");
19 | properties = new Properties();
20 | try {
21 | properties.load(inputStream);
22 | } catch (Exception e) {
23 | LOG.warn("No version information available", e);
24 | } finally {
25 | if (inputStream != null) {
26 | try {
27 | inputStream.close();
28 | } catch (IOException e) {
29 | // Ignore
30 | }
31 | }
32 | }
33 | }
34 |
35 | public static String getBuildNumber() {
36 | return properties.getProperty("gitrev");
37 | }
38 | }
39 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/ProxyConfig.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs;
2 |
3 | /** */
4 | public class ProxyConfig {
5 | public static final String MOUNT_TABLE_ZK_QUORUM = "dfs.nnproxy.mount-table.zk.quorum";
6 | public static final String MOUNT_TABLE_ZK_PATH = "dfs.nnproxy.mount-table.zk.path";
7 | public static final String MOUNT_TABLE_ZK_SESSION_TIMEOUT = "dfs.nnproxy.mount-table.zk.session.timeout";
8 | public static final int MOUNT_TABLE_ZK_SESSION_TIMEOUT_DEFAULT = 30000;
9 | public static final String MOUNT_TABLE_ZK_CONNECTION_TIMEOUT = "dfs.nnproxy.mount-table.zk.connection.timeout";
10 | public static final int MOUNT_TABLE_ZK_CONNECTION_TIMEOUT_DEFAULT = 30000;
11 | public static final String MOUNT_TABLE_ZK_MAX_RETRIES = "dfs.nnproxy.mount-table.zk.max.retries";
12 | public static final int MOUNT_TABLE_ZK_MAX_RETRIES_DEFAULT = 10;
13 | public static final String MOUNT_TABLE_ZK_RETRY_BASE_SLEEP = "dfs.nnproxy.mount-table.zk.retry.base-sleep";
14 | public static final int MOUNT_TABLE_ZK_RETRY_BASE_SLEEP_DEFAULT = 1000;
15 | public static final String PROXY_HANDLER_COUNT = "dfs.nnproxy.handler.count";
16 | public static final int PROXY_HANDLER_COUNT_DEFAULT = 2048;
17 | public static final String USER_PROXY_EXPIRE_MS = "dfs.nnproxy.user-proxy.expire.ms";
18 | public static final long USER_PROXY_EXPIRE_MS_DEFAULT = 3 * 3600 * 1000L;
19 | public static final String RPC_PORT = "dfs.nnproxy.rpc.port";
20 | public static final int RPC_PORT_DEFAULT = 65212;
21 | public static final String MAX_CONCURRENT_REQUEST_PER_FS = "dfs.nnproxy.max.concurrent.request-per-fs";
22 | public static final long MAX_CONCURRENT_REQUEST_PER_FS_DEFAULT = 1637;
23 | public static final String CACHE_REGISTRY_RELOAD_INTERVAL_MS = "dfs.nnproxy.cache.registry.reload-interval-ms";
24 | public static final long CACHE_REGISTRY_RELOAD_INTERVAL_MS_DEFAULT = 300 * 1000L;
25 | public static final String SUPERUSER = "dfs.nnproxy.superuser";
26 | public static final String SUPERUSER_DEFAULT = System.getProperty("user.name");
27 | }
28 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/server/NNProxy.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.server;
2 |
3 | import com.bytedance.hadoop.hdfs.BDManifest;
4 | import com.bytedance.hadoop.hdfs.server.cache.CacheRegistry;
5 | import com.bytedance.hadoop.hdfs.server.mount.BlockPoolRegistry;
6 | import com.bytedance.hadoop.hdfs.server.mount.MountsManager;
7 | import com.bytedance.hadoop.hdfs.server.proxy.ProxyMetrics;
8 | import com.bytedance.hadoop.hdfs.server.proxy.ProxyServer;
9 | import com.bytedance.hadoop.hdfs.server.proxy.RpcInvocationProxy;
10 | import com.bytedance.hadoop.hdfs.server.upstream.UpstreamManager;
11 | import com.google.common.annotations.VisibleForTesting;
12 | import org.apache.hadoop.classification.InterfaceAudience;
13 | import org.apache.hadoop.classification.InterfaceStability;
14 | import org.apache.hadoop.conf.Configuration;
15 | import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
16 | import org.slf4j.Logger;
17 | import org.slf4j.LoggerFactory;
18 |
19 | import java.io.IOException;
20 | import java.net.InetSocketAddress;
21 |
22 | @InterfaceAudience.Private
23 | @InterfaceStability.Stable
24 | public class NNProxy {
25 |
26 | private static final Logger LOG = LoggerFactory.getLogger(NNProxy.class);
27 |
28 | protected final Configuration conf;
29 | @VisibleForTesting
30 | protected MountsManager mounts;
31 | protected final UpstreamManager upstreamManager;
32 | protected final BlockPoolRegistry blockPoolRegistry;
33 | protected final CacheRegistry cacheRegistry;
34 | protected final RpcInvocationProxy router;
35 | protected final ProxyServer server;
36 |
37 | public static ProxyMetrics proxyMetrics;
38 |
39 | public NNProxy(Configuration conf) throws Exception {
40 | DefaultMetricsSystem.initialize("NNProxy");
41 | proxyMetrics = ProxyMetrics.create(conf);
42 | this.conf = conf;
43 | this.mounts = new MountsManager();
44 |
45 | this.upstreamManager = new UpstreamManager(this, conf);
46 | this.blockPoolRegistry = new BlockPoolRegistry(this, conf, upstreamManager);
47 | this.cacheRegistry = new CacheRegistry(this, conf, upstreamManager);
48 | this.router = new RpcInvocationProxy(this, conf, upstreamManager);
49 |
50 | this.server = new ProxyServer(this, conf, router);
51 | }
52 |
53 | public void start() throws IOException, InterruptedException {
54 | this.mounts.init(conf);
55 | this.mounts.start();
56 | this.mounts.waitUntilInstalled();
57 | Runtime.getRuntime().addShutdownHook(new Thread() {
58 | public void run() {
59 | shutdown();
60 | }
61 | });
62 | this.cacheRegistry.start();
63 | this.server.start();
64 | LOG.info("Started nnproxy, revision " + BDManifest.getBuildNumber());
65 | }
66 |
67 | public void shutdown() {
68 | this.cacheRegistry.shutdown();
69 | LOG.info("Gracefully shutting down nnproxy...");
70 | this.router.shutdown();
71 | this.server.shutdown();
72 | LOG.info("NNProxy shutdown completed");
73 | }
74 |
75 | public void join() throws InterruptedException {
76 | this.server.join();
77 | }
78 |
79 | public MountsManager getMounts() {
80 | return mounts;
81 | }
82 |
83 | public Configuration getConf() {
84 | return conf;
85 | }
86 |
87 | public UpstreamManager getUpstreamManager() {
88 | return upstreamManager;
89 | }
90 |
91 | public BlockPoolRegistry getBlockPoolRegistry() {
92 | return blockPoolRegistry;
93 | }
94 |
95 | public CacheRegistry getCacheRegistry() {
96 | return cacheRegistry;
97 | }
98 |
99 | public RpcInvocationProxy getRouter() {
100 | return router;
101 | }
102 |
103 | public ProxyServer getServer() {
104 | return server;
105 | }
106 |
107 | public InetSocketAddress getRpcAddress() {
108 | return server.getRpcAddress();
109 | }
110 |
111 | }
112 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/server/ProxyMain.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.server;
2 |
3 | import org.apache.hadoop.classification.InterfaceAudience;
4 | import org.apache.hadoop.classification.InterfaceStability;
5 | import org.apache.hadoop.conf.Configuration;
6 | import org.apache.hadoop.hdfs.HdfsConfiguration;
7 | import org.apache.hadoop.util.Tool;
8 | import org.apache.hadoop.util.ToolRunner;
9 | import org.slf4j.Logger;
10 | import org.slf4j.LoggerFactory;
11 |
12 | @InterfaceAudience.Private
13 | @InterfaceStability.Stable
14 | public class ProxyMain implements Tool {
15 |
16 | private static final Logger LOG = LoggerFactory.getLogger(ProxyMain.class);
17 |
18 | Configuration conf;
19 |
20 | public static void main(String[] args) throws Exception {
21 | ProxyMain main = new ProxyMain();
22 | System.exit(ToolRunner.run(new HdfsConfiguration(), main, args));
23 | }
24 |
25 | @Override
26 | public int run(String[] args) throws Exception {
27 | NNProxy nnProxy = new NNProxy(conf);
28 | nnProxy.start();
29 | nnProxy.join();
30 | LOG.info("NNProxy halted");
31 | return 0;
32 | }
33 |
34 | @Override
35 | public void setConf(Configuration conf) {
36 | this.conf = conf;
37 | }
38 |
39 | @Override
40 | public Configuration getConf() {
41 | return conf;
42 | }
43 | }
44 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/server/cache/CacheRegistry.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.server.cache;
2 |
3 | import com.bytedance.hadoop.hdfs.ProxyConfig;
4 | import com.bytedance.hadoop.hdfs.server.NNProxy;
5 | import com.bytedance.hadoop.hdfs.server.upstream.UpstreamManager;
6 | import com.google.common.collect.ImmutableList;
7 | import org.apache.hadoop.classification.InterfaceAudience;
8 | import org.apache.hadoop.classification.InterfaceStability;
9 | import org.apache.hadoop.conf.Configuration;
10 | import org.apache.hadoop.fs.BatchedRemoteIterator;
11 | import org.apache.hadoop.fs.CacheFlag;
12 | import org.apache.hadoop.fs.InvalidRequestException;
13 | import org.apache.hadoop.hdfs.DFSUtil;
14 | import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
15 | import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
16 | import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
17 | import org.slf4j.Logger;
18 | import org.slf4j.LoggerFactory;
19 |
20 | import java.io.IOException;
21 | import java.util.*;
22 | import java.util.concurrent.ExecutionException;
23 |
24 | import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
25 |
26 | /**
27 | * This manages a view of cache pools and directives aggregated from all backend NameNodes.
28 | * View is always updated in async fashion, thus view may be inconsistent after update.
29 | * Note that CachePool id may be indistinguishable between NameNodes.
30 | * To solve the complex, each id is marked with FsId on higher 16 bits.
31 | */
32 | @InterfaceAudience.Private
33 | @InterfaceStability.Evolving
34 | public class CacheRegistry {
35 |
36 | private static final Logger LOG = LoggerFactory.getLogger(CacheRegistry.class);
37 |
38 | final NNProxy nnProxy;
39 | final UpstreamManager upstreamManager;
40 | TreeMap directivesById =
41 | new TreeMap<>();
42 | TreeMap cachePools =
43 | new TreeMap<>();
44 | Map pool2fs = new HashMap<>();
45 | final String superuser;
46 | final int maxListCachePoolsResponses;
47 | final int maxListCacheDirectivesNumResponses;
48 | final long reloadIntervalMs;
49 | final Thread reloadThread;
50 | volatile boolean running;
51 |
52 | long maskDirectiveId(long id, long fsIndex) {
53 | id &= 0x0000ffffffffffffL;
54 | id |= (fsIndex << 48);
55 | return id;
56 | }
57 |
58 | long getFsIndex(long maskedId) {
59 | return maskedId >> 48;
60 | }
61 |
62 | long getDirectiveId(long maskedId) {
63 | return maskedId & 0x0000ffffffffffffL;
64 | }
65 |
66 | public CacheRegistry(NNProxy proxy, Configuration conf, UpstreamManager upstreamManager) {
67 | this.nnProxy = proxy;
68 | this.upstreamManager = upstreamManager;
69 | this.superuser = conf.get(ProxyConfig.SUPERUSER, ProxyConfig.SUPERUSER_DEFAULT);
70 |
71 | this.maxListCachePoolsResponses = conf.getInt(
72 | DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES,
73 | DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES_DEFAULT);
74 | this.maxListCacheDirectivesNumResponses = conf.getInt(
75 | DFS_NAMENODE_LIST_CACHE_DIRECTIVES_NUM_RESPONSES,
76 | DFS_NAMENODE_LIST_CACHE_DIRECTIVES_NUM_RESPONSES_DEFAULT);
77 | this.reloadIntervalMs = conf.getLong(
78 | ProxyConfig.CACHE_REGISTRY_RELOAD_INTERVAL_MS,
79 | ProxyConfig.CACHE_REGISTRY_RELOAD_INTERVAL_MS_DEFAULT);
80 | this.reloadThread = new Thread(new Runnable() {
81 | @Override
82 | public void run() {
83 | reloader();
84 | }
85 | });
86 | this.reloadThread.setName("Cache Registry Reloader");
87 | this.reloadThread.setDaemon(true);
88 | }
89 |
90 | public void start() {
91 | this.running = true;
92 | this.reloadThread.start();
93 | }
94 |
95 | public void shutdown() {
96 | this.running = false;
97 | this.reloadThread.interrupt();
98 | }
99 |
100 | List getAllCacheDirectives(UpstreamManager.Upstream upstream) throws IOException {
101 | CacheDirectiveInfo filter = new CacheDirectiveInfo.Builder().build();
102 | List directives = new ArrayList<>();
103 | long prevId = -1;
104 | while (true) {
105 | BatchedRemoteIterator.BatchedEntries it =
106 | upstream.protocol.listCacheDirectives(prevId, filter);
107 | if (it.size() == 0) {
108 | break;
109 | }
110 | for (int i = 0; i < it.size(); i++) {
111 | CacheDirectiveEntry entry = it.get(i);
112 | prevId = entry.getInfo().getId();
113 | directives.add(entry);
114 | }
115 | }
116 | return directives;
117 | }
118 |
119 | List getAllCachePools(UpstreamManager.Upstream upstream) throws IOException {
120 | String prevPool = "";
121 | List pools = new ArrayList<>();
122 |
123 | while (true) {
124 | BatchedRemoteIterator.BatchedEntries it = upstream.protocol.listCachePools(prevPool);
125 | if (it.size() == 0) {
126 | break;
127 | }
128 | for (int i = 0; i < it.size(); i++) {
129 | CachePoolEntry entry = it.get(i);
130 | prevPool = entry.getInfo().getPoolName();
131 | pools.add(entry);
132 | }
133 | }
134 | return pools;
135 | }
136 |
137 | List maskWithFsIndex(List entries, int fsIndex) {
138 | List masked = new ArrayList<>(entries.size());
139 | for (CacheDirectiveEntry entry : entries) {
140 | CacheDirectiveInfo info = new CacheDirectiveInfo.Builder()
141 | .setId(maskDirectiveId(entry.getInfo().getId(), fsIndex))
142 | .setPath(entry.getInfo().getPath())
143 | .setReplication(entry.getInfo().getReplication())
144 | .setPool(entry.getInfo().getPool())
145 | .setExpiration(entry.getInfo().getExpiration())
146 | .build();
147 | masked.add(new CacheDirectiveEntry(info, entry.getStats()));
148 | }
149 | return masked;
150 | }
151 |
152 | void reload() throws Exception {
153 | List allDirectives = new ArrayList<>();
154 | List allPools = new ArrayList<>();
155 | Map newPool2fs = new HashMap<>();
156 | int i = 0;
157 | for (String fs : nnProxy.getMounts().getAllFs()) {
158 | UpstreamManager.Upstream upstream = upstreamManager.getUpstream(superuser, fs);
159 | List pools = getAllCachePools(upstream);
160 | for (CachePoolEntry pool : pools) {
161 | newPool2fs.put(pool.getInfo().getPoolName(), fs);
162 | }
163 | allPools.addAll(pools);
164 | allDirectives.addAll(maskWithFsIndex(getAllCacheDirectives(upstream), i));
165 | i++;
166 | }
167 | TreeMap newDirectivesById =
168 | new TreeMap<>();
169 | TreeMap newCachePools =
170 | new TreeMap<>();
171 | for (CacheDirectiveEntry directive : allDirectives) {
172 | newDirectivesById.put(directive.getInfo().getId(), directive);
173 | }
174 | for (CachePoolEntry pool : allPools) {
175 | newCachePools.put(pool.getInfo().getPoolName(), pool);
176 | }
177 | LOG.debug("Cache directives: {}", newDirectivesById);
178 | LOG.debug("Cache pools: {}", newCachePools);
179 | LOG.debug("Cache pool to fs mapping: {}", newPool2fs);
180 | this.directivesById = newDirectivesById;
181 | this.cachePools = newCachePools;
182 | this.pool2fs = newPool2fs;
183 | }
184 |
185 | void reloader() {
186 | while (this.running) {
187 | try {
188 | reload();
189 | } catch (Exception e) {
190 | LOG.error("Failed to reload cache view", e);
191 | }
192 | try {
193 | Thread.sleep(reloadIntervalMs);
194 | } catch (InterruptedException e) {
195 | continue;
196 | }
197 | }
198 | }
199 |
200 | private static String validatePath(CacheDirectiveInfo directive)
201 | throws InvalidRequestException {
202 | if (directive.getPath() == null) {
203 | throw new InvalidRequestException("No path specified.");
204 | }
205 | String path = directive.getPath().toUri().getPath();
206 | if (!DFSUtil.isValidName(path)) {
207 | throw new InvalidRequestException("Invalid path '" + path + "'.");
208 | }
209 | return path;
210 | }
211 |
212 | public BatchedRemoteIterator.BatchedListEntries listCacheDirectives(long prevId,
213 | CacheDirectiveInfo filter) throws InvalidRequestException {
214 | final int NUM_PRE_ALLOCATED_ENTRIES = 16;
215 | String filterPath = null;
216 | if (filter.getPath() != null) {
217 | filterPath = validatePath(filter);
218 | }
219 | if (filter.getReplication() != null) {
220 | throw new InvalidRequestException(
221 | "Filtering by replication is unsupported.");
222 | }
223 |
224 | // Querying for a single ID
225 | final Long id = filter.getId();
226 | if (id != null) {
227 | if (!directivesById.containsKey(id)) {
228 | throw new InvalidRequestException("Did not find requested id " + id);
229 | }
230 | // Since we use a tailMap on directivesById, setting prev to id-1 gets
231 | // us the directive with the id (if present)
232 | prevId = id - 1;
233 | }
234 |
235 | ArrayList replies =
236 | new ArrayList(NUM_PRE_ALLOCATED_ENTRIES);
237 | int numReplies = 0;
238 | SortedMap tailMap =
239 | directivesById.tailMap(prevId + 1);
240 | for (Map.Entry cur : tailMap.entrySet()) {
241 | if (numReplies >= maxListCacheDirectivesNumResponses) {
242 | return new BatchedRemoteIterator.BatchedListEntries<>(replies, true);
243 | }
244 | CacheDirectiveInfo info = cur.getValue().getInfo();
245 |
246 | // If the requested ID is present, it should be the first item.
247 | // Hitting this case means the ID is not present, or we're on the second
248 | // item and should break out.
249 | if (id != null &&
250 | !(info.getId().equals(id))) {
251 | break;
252 | }
253 | if (filter.getPool() != null &&
254 | !info.getPool().equals(filter.getPool())) {
255 | continue;
256 | }
257 | if (filterPath != null &&
258 | !info.getPath().toUri().getPath().equals(filterPath)) {
259 | continue;
260 | }
261 | replies.add(cur.getValue());
262 | numReplies++;
263 | }
264 | return new BatchedRemoteIterator.BatchedListEntries<>(replies, false);
265 | }
266 |
267 | public BatchedRemoteIterator.BatchedListEntries listCachePools(String prevKey) {
268 | final int NUM_PRE_ALLOCATED_ENTRIES = 16;
269 | ArrayList results =
270 | new ArrayList(NUM_PRE_ALLOCATED_ENTRIES);
271 | SortedMap tailMap = cachePools.tailMap(prevKey, false);
272 | int numListed = 0;
273 | for (Map.Entry cur : tailMap.entrySet()) {
274 | if (numListed++ >= maxListCachePoolsResponses) {
275 | return new BatchedRemoteIterator.BatchedListEntries<>(results, true);
276 | }
277 | results.add(cur.getValue());
278 | }
279 | return new BatchedRemoteIterator.BatchedListEntries<>(results, false);
280 | }
281 |
282 | UpstreamManager.Upstream getUpstream(String pool) throws IOException {
283 | String fs = pool2fs.get(pool);
284 | int fsIndex = -1;
285 | if (fs == null) {
286 | throw new IOException("Cannot find namespace associated with pool " + pool);
287 | }
288 | ImmutableList allFs = nnProxy.getMounts().getAllFs();
289 | for (int i = 0; i < allFs.size(); i++) {
290 | if (allFs.get(i).equals(fs)) {
291 | fsIndex = i;
292 | break;
293 | }
294 | }
295 | if (fsIndex < 0) {
296 | throw new IOException("No fs index associated with fs " + fs);
297 | }
298 | try {
299 | UpstreamManager.Upstream upstream = upstreamManager.getUpstream(superuser, fs);
300 | upstream.setFsIndex(fsIndex);
301 | return upstream;
302 | } catch (ExecutionException e) {
303 | throw new IOException("Failed to get upstream");
304 | }
305 | }
306 |
307 | public long addCacheDirective(CacheDirectiveInfo directive, EnumSet flags)
308 | throws IOException {
309 | UpstreamManager.Upstream upstream = getUpstream(directive.getPool());
310 | long id = maskDirectiveId(upstream.protocol.addCacheDirective(directive, flags), upstream.fsIndex);
311 | reloadThread.interrupt();
312 | return id;
313 | }
314 |
315 | public void modifyCacheDirective(CacheDirectiveInfo directive, EnumSet flags)
316 | throws IOException {
317 | UpstreamManager.Upstream upstream = getUpstream(directive.getPool());
318 | upstream.protocol.modifyCacheDirective(directive, flags);
319 | reloadThread.interrupt();
320 | }
321 |
322 | public void removeCacheDirective(long id) throws IOException {
323 | int fsIndex = (int) getFsIndex(id);
324 | long directiveId = getDirectiveId(id);
325 | ImmutableList allFs = nnProxy.getMounts().getAllFs();
326 | if (allFs.size() <= fsIndex) {
327 | throw new IOException("No fs associated with index " + fsIndex);
328 | }
329 | UpstreamManager.Upstream upstream;
330 | try {
331 | upstream = upstreamManager.getUpstream(superuser, allFs.get(fsIndex));
332 | } catch (ExecutionException e) {
333 | throw new IOException("Failed to get upstream");
334 | }
335 | upstream.protocol.removeCacheDirective(directiveId);
336 | reloadThread.interrupt();
337 | }
338 | }
339 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/server/exception/WrappedExecutionException.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.server.exception;
2 |
3 | import org.apache.hadoop.classification.InterfaceAudience;
4 | import org.apache.hadoop.classification.InterfaceStability;
5 |
6 | /**
7 | * An ExecutionException wrapped as unchecked, this is for internal exception handling in proxy
8 | */
9 | @InterfaceAudience.Private
10 | @InterfaceStability.Stable
11 | public class WrappedExecutionException extends RuntimeException {
12 | public WrappedExecutionException() {
13 | }
14 |
15 | public WrappedExecutionException(String message) {
16 | super(message);
17 | }
18 |
19 | public WrappedExecutionException(String message, Throwable cause) {
20 | super(message, cause);
21 | }
22 |
23 | public WrappedExecutionException(Throwable cause) {
24 | super(cause);
25 | }
26 | }
27 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/server/mount/BlockPoolRegistry.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.server.mount;
2 |
3 | import com.bytedance.hadoop.hdfs.ProxyConfig;
4 | import com.bytedance.hadoop.hdfs.server.NNProxy;
5 | import com.bytedance.hadoop.hdfs.server.upstream.UpstreamManager;
6 | import org.apache.hadoop.classification.InterfaceAudience;
7 | import org.apache.hadoop.classification.InterfaceStability;
8 | import org.apache.hadoop.conf.Configuration;
9 | import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
10 | import org.slf4j.Logger;
11 | import org.slf4j.LoggerFactory;
12 |
13 | import java.io.IOException;
14 | import java.util.HashMap;
15 | import java.util.Map;
16 | import java.util.concurrent.ExecutionException;
17 |
18 | /**
19 | * Provides blockPoolId to NameNode mapping.
20 | * This is based on the assumption that blockPoolId assigned for one particular FS never changes.
21 | */
22 | @InterfaceAudience.Private
23 | @InterfaceStability.Stable
24 | public class BlockPoolRegistry {
25 |
26 | private static final Logger LOG = LoggerFactory.getLogger(BlockPoolRegistry.class);
27 |
28 | final NNProxy nnProxy;
29 | final UpstreamManager upstreamManager;
30 | final Map bp2fs;
31 | final String superuser;
32 |
33 | public BlockPoolRegistry(NNProxy proxy, Configuration conf, UpstreamManager upstreamManager) {
34 | this.nnProxy = proxy;
35 | this.upstreamManager = upstreamManager;
36 | this.bp2fs = new HashMap<>();
37 | this.superuser = conf.get(ProxyConfig.SUPERUSER, ProxyConfig.SUPERUSER_DEFAULT);
38 | }
39 |
40 | void refreshBlockPools() throws ExecutionException, IOException {
41 | for (String fs : nnProxy.getMounts().getAllFs()) {
42 | NamespaceInfo nsInfo = upstreamManager.getUpstream(superuser, fs).nnProxyAndInfo.getProxy().versionRequest();
43 | String bpId = nsInfo.getBlockPoolID();
44 | bp2fs.put(bpId, fs);
45 | }
46 | }
47 |
48 | public synchronized String getFs(String bpId) throws IOException {
49 | if (bp2fs.containsKey(bpId)) {
50 | return bp2fs.get(bpId);
51 | }
52 | try {
53 | refreshBlockPools();
54 | } catch (ExecutionException e) {
55 | LOG.error("Failed to refresh block pools", e);
56 | }
57 | return bp2fs.get(bpId);
58 | }
59 |
60 | }
61 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/server/mount/MountsManager.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.server.mount;
2 |
3 | import com.bytedance.hadoop.hdfs.ProxyConfig;
4 | import com.google.common.annotations.VisibleForTesting;
5 | import com.google.common.collect.ImmutableList;
6 | import org.apache.commons.lang.StringUtils;
7 | import org.apache.curator.framework.CuratorFramework;
8 | import org.apache.curator.framework.CuratorFrameworkFactory;
9 | import org.apache.curator.framework.recipes.cache.NodeCache;
10 | import org.apache.curator.framework.recipes.cache.NodeCacheListener;
11 | import org.apache.curator.retry.ExponentialBackoffRetry;
12 | import org.apache.hadoop.classification.InterfaceAudience;
13 | import org.apache.hadoop.classification.InterfaceStability;
14 | import org.apache.hadoop.conf.Configuration;
15 | import org.apache.hadoop.service.AbstractService;
16 | import org.slf4j.Logger;
17 | import org.slf4j.LoggerFactory;
18 |
19 | import java.util.ArrayList;
20 | import java.util.Arrays;
21 | import java.util.List;
22 |
23 | /**
24 | * Manages mount table and keep up-to-date to ZooKeeper.
25 | */
26 | @InterfaceAudience.Private
27 | @InterfaceStability.Stable
28 | public class MountsManager extends AbstractService {
29 |
30 | private static final Logger LOG = LoggerFactory.getLogger(MountsManager.class);
31 |
32 | static class MountEntry {
33 | final String fsUri;
34 | final String mountPoint;
35 | final String[] attributes;
36 |
37 | public MountEntry(String fsUri, String mountPoint, String[] attributes) {
38 | this.fsUri = fsUri;
39 | this.mountPoint = mountPoint;
40 | this.attributes = attributes;
41 | }
42 |
43 | @Override
44 | public String toString() {
45 | return "MountEntry [" +
46 | "fsUri=" + fsUri +
47 | ", mountPoint=" + mountPoint +
48 | ", attributes=" + Arrays.toString(attributes) +
49 | ']';
50 | }
51 | }
52 |
53 | CuratorFramework framework;
54 | String zkMountTablePath;
55 | ImmutableList mounts;
56 | ImmutableList allFs;
57 | MountEntry root;
58 | NodeCache nodeCache;
59 |
60 | @VisibleForTesting
61 | protected volatile boolean installed;
62 |
63 | public MountsManager() {
64 | super("MountsManager");
65 | }
66 |
67 | @Override
68 | protected void serviceInit(Configuration conf) throws Exception {
69 | super.serviceInit(conf);
70 | String zkConnectString = conf.get(ProxyConfig.MOUNT_TABLE_ZK_QUORUM);
71 | zkMountTablePath = conf.get(ProxyConfig.MOUNT_TABLE_ZK_PATH);
72 | int sessionTimeout = conf.getInt(ProxyConfig.MOUNT_TABLE_ZK_SESSION_TIMEOUT,
73 | ProxyConfig.MOUNT_TABLE_ZK_SESSION_TIMEOUT_DEFAULT);
74 | int connectionTimeout = conf.getInt(ProxyConfig.MOUNT_TABLE_ZK_CONNECTION_TIMEOUT,
75 | ProxyConfig.MOUNT_TABLE_ZK_CONNECTION_TIMEOUT_DEFAULT);
76 | int maxRetries = conf.getInt(ProxyConfig.MOUNT_TABLE_ZK_MAX_RETRIES,
77 | ProxyConfig.MOUNT_TABLE_ZK_MAX_RETRIES_DEFAULT);
78 | int retryBaseSleep = conf.getInt(ProxyConfig.MOUNT_TABLE_ZK_RETRY_BASE_SLEEP,
79 | ProxyConfig.MOUNT_TABLE_ZK_RETRY_BASE_SLEEP_DEFAULT);
80 | framework = CuratorFrameworkFactory.newClient(
81 | zkConnectString, sessionTimeout, connectionTimeout,
82 | new ExponentialBackoffRetry(retryBaseSleep, maxRetries));
83 | installed = false;
84 | }
85 |
86 | public ImmutableList getMounts() {
87 | return mounts;
88 | }
89 |
90 | public ImmutableList getAllFs() {
91 | return allFs;
92 | }
93 |
94 | public String resolve(String path) {
95 | ImmutableList entries = this.mounts;
96 | MountEntry chosen = null;
97 | for (MountEntry entry : entries) {
98 | if (path == null || !(path.startsWith(entry.mountPoint + "/") || path.equals(entry.mountPoint))) {
99 | continue;
100 | }
101 | if (chosen == null || chosen.mountPoint.length() < entry.mountPoint.length()) {
102 | chosen = entry;
103 | }
104 | }
105 | if (chosen == null) {
106 | chosen = root;
107 | }
108 | return chosen.fsUri;
109 | }
110 |
111 | /**
112 | * Determine whether given path is exactly a valid mount point
113 | *
114 | * @param path
115 | * @return
116 | */
117 | public boolean isMountPoint(String path) {
118 | ImmutableList entries = this.mounts;
119 | for (MountEntry entry : entries) {
120 | if (entry.mountPoint.equals(path)) {
121 | return true;
122 | }
123 | }
124 | return false;
125 | }
126 |
127 | /**
128 | * Determine whether given path contains a mount point.
129 | * Directory is considered unified even if itself is a mount point, unless it contains another mount point.
130 | *
131 | * @param path
132 | * @return
133 | */
134 | public boolean isUnified(String path) {
135 | String prefix = path + "/";
136 | ImmutableList entries = this.mounts;
137 | for (MountEntry entry : entries) {
138 | if (entry.mountPoint.startsWith(prefix)) {
139 | return false;
140 | }
141 | }
142 | return true;
143 | }
144 |
145 | @VisibleForTesting
146 | protected void installMountTable(List entries) {
147 | LOG.info("Installed mount table: " + entries);
148 | List fs = new ArrayList<>();
149 | for (MountEntry entry : entries) {
150 | if (entry.mountPoint.equals("/")) {
151 | root = entry;
152 | }
153 | if (!fs.contains(entry.fsUri)) {
154 | fs.add(entry.fsUri);
155 | }
156 | }
157 | this.allFs = ImmutableList.copyOf(fs);
158 | this.mounts = ImmutableList.copyOf(entries);
159 | this.installed = true;
160 | }
161 |
162 | @VisibleForTesting
163 | protected List parseMountTable(String mounts) {
164 | List table = new ArrayList<>();
165 | boolean hasRoot = false;
166 | for (String s : mounts.split("\n")) {
167 | if (StringUtils.isEmpty(s)) {
168 | continue;
169 | }
170 | String[] cols = s.split(" ");
171 | String fsUri = cols[0];
172 | String mountPoint = cols[1];
173 | String[] attrs = (cols.length > 2) ? cols[2].split(",") : new String[0];
174 | table.add(new MountEntry(fsUri, mountPoint, attrs));
175 | if (mountPoint.equals("/")) {
176 | hasRoot = true;
177 | }
178 | }
179 | if (!hasRoot) {
180 | LOG.error("Ignored invalid mount table: " + mounts);
181 | return null;
182 | }
183 | return table;
184 | }
185 |
186 | @VisibleForTesting
187 | protected void handleMountTableChange(byte[] data) {
188 | if (data == null || data.length == 0) {
189 | LOG.info("Invalid mount table");
190 | return;
191 | }
192 | String mounts = new String(data);
193 | List table = parseMountTable(mounts);
194 | if (table != null) {
195 | installMountTable(table);
196 | }
197 | }
198 |
199 | @Override
200 | protected void serviceStart() throws Exception {
201 | framework.start();
202 | nodeCache = new NodeCache(framework, zkMountTablePath, false);
203 | nodeCache.getListenable().addListener(new NodeCacheListener() {
204 | @Override
205 | public void nodeChanged() throws Exception {
206 | handleMountTableChange(nodeCache.getCurrentData().getData());
207 | }
208 | });
209 | nodeCache.start(false);
210 | }
211 |
212 | @Override
213 | protected void serviceStop() throws Exception {
214 | nodeCache.close();
215 | framework.close();
216 | }
217 |
218 | public void waitUntilInstalled() throws InterruptedException {
219 | while (!installed) {
220 | Thread.sleep(100);
221 | }
222 | }
223 |
224 | public String dump() {
225 | ImmutableList entries = this.mounts;
226 | StringBuilder result = new StringBuilder();
227 | for (MountEntry entry : entries) {
228 | result.append(entry.fsUri);
229 | result.append(' ');
230 | result.append(entry.mountPoint);
231 | result.append(' ');
232 | result.append(StringUtils.join(entry.attributes, ","));
233 | result.append('\n');
234 | }
235 | return result.toString();
236 | }
237 |
238 | public void load(String mounts) throws Exception {
239 | framework.setData().forPath(zkMountTablePath, mounts.getBytes());
240 | }
241 | }
242 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/server/proxy/ProxyClientProtocolHandler.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.server.proxy;
2 |
3 | import com.bytedance.hadoop.hdfs.server.NNProxy;
4 | import com.bytedance.hadoop.hdfs.server.upstream.UpstreamManager;
5 | import org.apache.hadoop.classification.InterfaceAudience;
6 | import org.apache.hadoop.classification.InterfaceStability;
7 | import org.apache.hadoop.conf.Configuration;
8 | import org.apache.hadoop.crypto.CryptoProtocolVersion;
9 | import org.apache.hadoop.fs.*;
10 | import org.apache.hadoop.fs.permission.AclEntry;
11 | import org.apache.hadoop.fs.permission.AclStatus;
12 | import org.apache.hadoop.fs.permission.FsAction;
13 | import org.apache.hadoop.fs.permission.FsPermission;
14 | import org.apache.hadoop.hdfs.inotify.EventBatchList;
15 | import org.apache.hadoop.hdfs.protocol.*;
16 | import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
17 | import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
18 | import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
19 | import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
20 | import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
21 | import org.apache.hadoop.io.EnumSetWritable;
22 | import org.apache.hadoop.io.Text;
23 | import org.apache.hadoop.security.AccessControlException;
24 | import org.apache.hadoop.security.token.Token;
25 | import org.slf4j.Logger;
26 | import org.slf4j.LoggerFactory;
27 |
28 | import java.io.FileNotFoundException;
29 | import java.io.IOException;
30 | import java.util.*;
31 |
32 | /** */
33 | @InterfaceAudience.Private
34 | @InterfaceStability.Evolving
35 | public class ProxyClientProtocolHandler implements ClientProtocol {
36 |
37 | private static final Logger LOG = LoggerFactory.getLogger(ProxyClientProtocolHandler.class);
38 |
39 | final NNProxy nnProxy;
40 | final Configuration conf;
41 | final UpstreamManager upstreamManager;
42 | final Router router;
43 |
44 | public ProxyClientProtocolHandler(NNProxy nnProxy, Configuration conf, UpstreamManager upstreamManager) {
45 | this.nnProxy = nnProxy;
46 | this.conf = conf;
47 | this.upstreamManager = upstreamManager;
48 | this.router = new Router(nnProxy, conf, upstreamManager);
49 | }
50 |
51 | void ensureCanRename(String path) throws IOException {
52 | if (nnProxy.getMounts().isMountPoint(path)) {
53 | throw new IOException("Cannot rename a mount point (" + path + ")");
54 | }
55 | if (!nnProxy.getMounts().isUnified(path)) {
56 | throw new IOException("Cannot rename a non-unified directory " + path + " (contains mount point)");
57 | }
58 | }
59 |
60 | /* begin protocol handlers */
61 |
62 | @Override
63 | public LocatedBlocks getBlockLocations(String src, long offset, long length)
64 | throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException {
65 | RouteInfo routeInfo = router.route(src);
66 | return routeInfo.upstream.getBlockLocations(routeInfo.realPath, offset, length);
67 | }
68 |
69 | @Override
70 | public FsServerDefaults getServerDefaults() throws IOException {
71 | return router.getRoot().upstream.getServerDefaults();
72 | }
73 |
74 | @Override
75 | public HdfsFileStatus create(String src, FsPermission masked, String clientName, EnumSetWritable flag, boolean createParent, short replication, long blockSize, CryptoProtocolVersion[] supportedVersions) throws AccessControlException, AlreadyBeingCreatedException, DSQuotaExceededException, FileAlreadyExistsException, FileNotFoundException, NSQuotaExceededException, ParentNotDirectoryException, SafeModeException, UnresolvedLinkException, SnapshotAccessControlException, IOException {
76 | RouteInfo routeInfo = router.route(src);
77 | return routeInfo.upstream.create(routeInfo.realPath, masked, clientName, flag, createParent, replication, blockSize, supportedVersions);
78 | }
79 |
80 | @Override
81 | public LocatedBlock append(String src, String clientName) throws AccessControlException, DSQuotaExceededException, FileNotFoundException, SafeModeException, UnresolvedLinkException, SnapshotAccessControlException, IOException {
82 | RouteInfo routeInfo = router.route(src);
83 | return routeInfo.upstream.append(routeInfo.realPath, clientName);
84 | }
85 |
86 | @Override
87 | public boolean setReplication(String src, short replication) throws AccessControlException, DSQuotaExceededException, FileNotFoundException, SafeModeException, UnresolvedLinkException, SnapshotAccessControlException, IOException {
88 | RouteInfo routeInfo = router.route(src);
89 | return routeInfo.upstream.setReplication(routeInfo.realPath, replication);
90 | }
91 |
92 | @Override
93 | public BlockStoragePolicy[] getStoragePolicies() throws IOException {
94 | return router.getRoot().upstream.getStoragePolicies();
95 | }
96 |
97 | @Override
98 | public void setStoragePolicy(String src, String policyName) throws SnapshotAccessControlException, UnresolvedLinkException, FileNotFoundException, QuotaExceededException, IOException {
99 | RouteInfo routeInfo = router.route(src);
100 | routeInfo.upstream.setStoragePolicy(routeInfo.realPath, policyName);
101 | }
102 |
103 | @Override
104 | public void setPermission(String src, FsPermission permission) throws
105 | AccessControlException, FileNotFoundException, SafeModeException, UnresolvedLinkException, SnapshotAccessControlException, IOException {
106 | RouteInfo routeInfo = router.route(src);
107 | routeInfo.upstream.setPermission(routeInfo.realPath, permission);
108 | }
109 |
110 | @Override
111 | public void setOwner(String src, String username, String groupname) throws AccessControlException, FileNotFoundException, SafeModeException, UnresolvedLinkException, SnapshotAccessControlException, IOException {
112 | RouteInfo routeInfo = router.route(src);
113 | routeInfo.upstream.setOwner(routeInfo.realPath, username, groupname);
114 | }
115 |
116 | @Override
117 | public void abandonBlock(ExtendedBlock b, long fileId, String src, String holder) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException {
118 | RouteInfo routeInfo = router.route(src);
119 | routeInfo.upstream.abandonBlock(b, fileId, routeInfo.realPath, holder);
120 | }
121 |
122 | @Override
123 | public LocatedBlock addBlock(String src, String clientName, ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId, String[] favoredNodes) throws AccessControlException, FileNotFoundException, NotReplicatedYetException, SafeModeException, UnresolvedLinkException, IOException {
124 | RouteInfo routeInfo = router.route(src);
125 | return routeInfo.upstream.addBlock(routeInfo.realPath, clientName, previous, excludeNodes, fileId, favoredNodes);
126 | }
127 |
128 | @Override
129 | public LocatedBlock getAdditionalDatanode(String src, long fileId, ExtendedBlock blk, DatanodeInfo[] existings, String[] existingStorageIDs, DatanodeInfo[] excludes, int numAdditionalNodes, String clientName) throws AccessControlException, FileNotFoundException, SafeModeException, UnresolvedLinkException, IOException {
130 | RouteInfo routeInfo = router.route(src);
131 | return routeInfo.upstream.getAdditionalDatanode(routeInfo.realPath, fileId, blk, existings, existingStorageIDs, excludes, numAdditionalNodes, clientName);
132 | }
133 |
134 | @Override
135 | public boolean complete(String src, String clientName, ExtendedBlock last, long fileId) throws AccessControlException, FileNotFoundException, SafeModeException, UnresolvedLinkException, IOException {
136 | RouteInfo routeInfo = router.route(src);
137 | return routeInfo.upstream.complete(routeInfo.realPath, clientName, last, fileId);
138 | }
139 |
140 | @Override
141 | public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
142 | Map> fsBlocks = new HashMap<>();
143 | for (LocatedBlock blk : blocks) {
144 | String bpId = blk.getBlock().getBlockPoolId();
145 | String fs = nnProxy.getBlockPoolRegistry().getFs(bpId);
146 | if (fs == null) {
147 | throw new IOException("Unknown block pool: " + bpId);
148 | }
149 | if (!fsBlocks.containsKey(fs)) {
150 | fsBlocks.put(fs, new ArrayList());
151 | }
152 | fsBlocks.get(fs).add(blk);
153 | }
154 | for (Map.Entry> entry : fsBlocks.entrySet()) {
155 | String fs = entry.getKey();
156 | router.getProtocol(fs).reportBadBlocks(entry.getValue().toArray(new LocatedBlock[0]));
157 | }
158 | }
159 |
160 | @Override
161 | public boolean rename(String src, String dst) throws UnresolvedLinkException, SnapshotAccessControlException, IOException {
162 | ensureCanRename(src);
163 | ensureCanRename(dst);
164 | RouteInfo srcRouteInfo = router.route(src);
165 | RouteInfo dstRouteInfo = router.route(dst);
166 | if (!srcRouteInfo.fs.equals(dstRouteInfo.fs)) {
167 | throw new IOException("Cannot rename across namespaces");
168 | }
169 | return srcRouteInfo.upstream.rename(srcRouteInfo.realPath, dstRouteInfo.realPath);
170 | }
171 |
172 | @Override
173 | public void concat(String trg, String[] srcs) throws IOException, UnresolvedLinkException, SnapshotAccessControlException {
174 | RouteInfo trgRouteInfo = router.route(trg);
175 | RouteInfo[] routeInfos = new RouteInfo[srcs.length];
176 | for (int i = 0; i < srcs.length; i++) {
177 | routeInfos[i] = router.route(srcs[i]);
178 | }
179 | String fs = null;
180 | String[] newSrcs = new String[srcs.length];
181 | for (int i = 0; i < routeInfos.length; i++) {
182 | if (fs != null && !fs.equals(routeInfos[i].fs)) {
183 | throw new IOException("Cannot concat across namespaces");
184 | }
185 | fs = routeInfos[i].fs;
186 | newSrcs[i] = routeInfos[i].realPath;
187 | }
188 | if (fs != null && !fs.equals(trgRouteInfo.fs)) {
189 | throw new IOException("Cannot concat across namespaces");
190 | }
191 | trgRouteInfo.upstream.concat(trgRouteInfo.realPath, newSrcs);
192 | }
193 |
194 | @Override
195 | public void rename2(String src, String dst, Options.Rename... options) throws AccessControlException, DSQuotaExceededException, FileAlreadyExistsException, FileNotFoundException, NSQuotaExceededException, ParentNotDirectoryException, SafeModeException, UnresolvedLinkException, SnapshotAccessControlException, IOException {
196 | ensureCanRename(src);
197 | ensureCanRename(dst);
198 | RouteInfo srcRouteInfo = router.route(src);
199 | RouteInfo dstRouteInfo = router.route(dst);
200 | if (!srcRouteInfo.fs.equals(dstRouteInfo.fs)) {
201 | throw new IOException("Cannot rename across namespaces");
202 | }
203 | srcRouteInfo.upstream.rename2(srcRouteInfo.realPath, dstRouteInfo.realPath, options);
204 | }
205 |
206 | @Override
207 | public boolean delete(String src, boolean recursive) throws AccessControlException, FileNotFoundException, SafeModeException, UnresolvedLinkException, SnapshotAccessControlException, IOException {
208 | RouteInfo routeInfo = router.route(src);
209 | return routeInfo.upstream.delete(routeInfo.realPath, recursive);
210 | }
211 |
212 | @Override
213 | public boolean mkdirs(String src, FsPermission masked, boolean createParent) throws AccessControlException, FileAlreadyExistsException, FileNotFoundException, NSQuotaExceededException, ParentNotDirectoryException, SafeModeException, UnresolvedLinkException, SnapshotAccessControlException, IOException {
214 | RouteInfo routeInfo = router.route(src);
215 | return routeInfo.upstream.mkdirs(routeInfo.realPath, masked, createParent);
216 | }
217 |
218 | @Override
219 | public DirectoryListing getListing(String src, byte[] startAfter, boolean needLocation) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException {
220 | RouteInfo routeInfo = router.route(src);
221 | return routeInfo.upstream.getListing(routeInfo.realPath, startAfter, needLocation);
222 | }
223 |
224 | @Override
225 | public SnapshottableDirectoryStatus[] getSnapshottableDirListing() throws IOException {
226 | return new SnapshottableDirectoryStatus[0];
227 | }
228 |
229 | @Override
230 | public void renewLease(String clientName) throws AccessControlException, IOException {
231 | // currently, just renew lease on all namenodes
232 | for (String fs : nnProxy.getMounts().getAllFs()) {
233 | router.getProtocol(fs).renewLease(clientName);
234 | }
235 | }
236 |
237 | @Override
238 | public boolean recoverLease(String src, String clientName) throws IOException {
239 | RouteInfo routeInfo = router.route(src);
240 | return routeInfo.upstream.recoverLease(routeInfo.realPath, clientName);
241 | }
242 |
243 | @Override
244 | public long[] getStats() throws IOException {
245 | return router.getRoot().upstream.getStats();
246 | }
247 |
248 | @Override
249 | public DatanodeInfo[] getDatanodeReport(HdfsConstants.DatanodeReportType type) throws IOException {
250 | throw new IOException("Invalid operation, do not use proxy");
251 | }
252 |
253 | @Override
254 | public DatanodeStorageReport[] getDatanodeStorageReport(HdfsConstants.DatanodeReportType type) throws IOException {
255 | throw new IOException("Invalid operation, do not use proxy");
256 | }
257 |
258 | @Override
259 | public long getPreferredBlockSize(String filename) throws IOException, UnresolvedLinkException {
260 | RouteInfo routeInfo = router.route(filename);
261 | return routeInfo.upstream.getPreferredBlockSize(routeInfo.realPath);
262 | }
263 |
264 | @Override
265 | public boolean setSafeMode(HdfsConstants.SafeModeAction action, boolean isChecked) throws IOException {
266 | if (action.equals(HdfsConstants.SafeModeAction.SAFEMODE_GET)) {
267 | // FIXME: properly handle
268 | return false;
269 | }
270 | throw new IOException("Invalid operation, do not use proxy");
271 | }
272 |
273 | @Override
274 | public void saveNamespace() throws AccessControlException, IOException {
275 | throw new IOException("Invalid operation, do not use proxy");
276 | }
277 |
278 | @Override
279 | public long rollEdits() throws AccessControlException, IOException {
280 | throw new IOException("Invalid operation, do not use proxy");
281 | }
282 |
283 | @Override
284 | public boolean restoreFailedStorage(String arg) throws AccessControlException, IOException {
285 | throw new IOException("Invalid operation, do not use proxy");
286 | }
287 |
288 | @Override
289 | public void refreshNodes() throws IOException {
290 | throw new IOException("Invalid operation, do not use proxy");
291 | }
292 |
293 | @Override
294 | public void finalizeUpgrade() throws IOException {
295 | throw new IOException("Invalid operation, do not use proxy");
296 | }
297 |
298 | @Override
299 | public RollingUpgradeInfo rollingUpgrade(HdfsConstants.RollingUpgradeAction action) throws IOException {
300 | throw new IOException("Invalid operation, do not use proxy");
301 | }
302 |
303 | @Override
304 | public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie) throws IOException {
305 | RouteInfo routeInfo = router.route(path);
306 | return routeInfo.upstream.listCorruptFileBlocks(routeInfo.realPath, cookie);
307 | }
308 |
309 | @Override
310 | public void metaSave(String filename) throws IOException {
311 | throw new IOException("Invalid operation, do not use proxy");
312 | }
313 |
314 | @Override
315 | public void setBalancerBandwidth(long bandwidth) throws IOException {
316 | throw new IOException("Invalid operation, do not use proxy");
317 | }
318 |
319 | @Override
320 | public HdfsFileStatus getFileInfo(String src) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException {
321 | RouteInfo routeInfo = router.route(src);
322 | return routeInfo.upstream.getFileInfo(routeInfo.realPath);
323 | }
324 |
325 | @Override
326 | public boolean isFileClosed(String src) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException {
327 | RouteInfo routeInfo = router.route(src);
328 | return routeInfo.upstream.isFileClosed(routeInfo.realPath);
329 | }
330 |
331 | @Override
332 | public HdfsFileStatus getFileLinkInfo(String src) throws AccessControlException, UnresolvedLinkException, IOException {
333 | RouteInfo routeInfo = router.route(src);
334 | return routeInfo.upstream.getFileInfo(routeInfo.realPath);
335 | }
336 |
337 | @Override
338 | public ContentSummary getContentSummary(String path) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException {
339 | RouteInfo routeInfo = router.route(path);
340 | return routeInfo.upstream.getContentSummary(routeInfo.realPath);
341 | }
342 |
343 | @Override
344 | public void setQuota(String path, long namespaceQuota, long diskspaceQuota) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, SnapshotAccessControlException, IOException {
345 | RouteInfo routeInfo = router.route(path);
346 | routeInfo.upstream.setQuota(routeInfo.realPath, namespaceQuota, diskspaceQuota);
347 | }
348 |
349 | @Override
350 | public void fsync(String src, long inodeId, String client, long lastBlockLength) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException {
351 | RouteInfo routeInfo = router.route(src);
352 | routeInfo.upstream.fsync(routeInfo.realPath, inodeId, client, lastBlockLength);
353 | }
354 |
355 | @Override
356 | public void setTimes(String src, long mtime, long atime) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, SnapshotAccessControlException, IOException {
357 | RouteInfo routeInfo = router.route(src);
358 | routeInfo.upstream.setTimes(routeInfo.realPath, mtime, atime);
359 | }
360 |
361 | @Override
362 | public void createSymlink(String target, String link, FsPermission dirPerm, boolean createParent) throws AccessControlException, FileAlreadyExistsException, FileNotFoundException, ParentNotDirectoryException, SafeModeException, UnresolvedLinkException, SnapshotAccessControlException, IOException {
363 | RouteInfo routeInfo = router.route(target);
364 | routeInfo.upstream.getFileInfo(routeInfo.realPath);
365 | }
366 |
367 | @Override
368 | public String getLinkTarget(String path) throws AccessControlException, FileNotFoundException, IOException {
369 | RouteInfo routeInfo = router.route(path);
370 | return routeInfo.upstream.getLinkTarget(routeInfo.realPath);
371 | }
372 |
373 | @Override
374 | public LocatedBlock updateBlockForPipeline(ExtendedBlock block, String clientName) throws IOException {
375 | return router.getUpstreamForBlockPool(block.getBlockPoolId()).updateBlockForPipeline(block, clientName);
376 | }
377 |
378 | @Override
379 | public void updatePipeline(String clientName, ExtendedBlock oldBlock, ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs) throws IOException {
380 | if (!newBlock.getBlockPoolId().equals(oldBlock.getBlockPoolId())) {
381 | throw new IOException("Cannot update pipeline across block pools");
382 | }
383 | router.getUpstreamForBlockPool(newBlock.getBlockPoolId()).updatePipeline(clientName, oldBlock, newBlock, newNodes, newStorageIDs);
384 | }
385 |
386 | @Override
387 | public Token getDelegationToken(Text renewer) throws IOException {
388 | throw new IOException("Invalid operation, do not use proxy");
389 | }
390 |
391 | @Override
392 | public long renewDelegationToken(Token token) throws IOException {
393 | throw new IOException("Invalid operation, do not use proxy");
394 | }
395 |
396 | @Override
397 | public void cancelDelegationToken(Token token) throws IOException {
398 | throw new IOException("Invalid operation, do not use proxy");
399 | }
400 |
401 | @Override
402 | public DataEncryptionKey getDataEncryptionKey() throws IOException {
403 | return router.getRoot().upstream.getDataEncryptionKey();
404 | }
405 |
406 | @Override
407 | public String createSnapshot(String snapshotRoot, String snapshotName) throws IOException {
408 | throw new IOException("Invalid operation, do not use proxy");
409 | }
410 |
411 | @Override
412 | public void deleteSnapshot(String snapshotRoot, String snapshotName) throws IOException {
413 | throw new IOException("Invalid operation, do not use proxy");
414 | }
415 |
416 | @Override
417 | public void renameSnapshot(String snapshotRoot, String snapshotOldName, String snapshotNewName) throws IOException {
418 | throw new IOException("Invalid operation, do not use proxy");
419 | }
420 |
421 | @Override
422 | public void allowSnapshot(String snapshotRoot) throws IOException {
423 | throw new IOException("Invalid operation, do not use proxy");
424 | }
425 |
426 | @Override
427 | public void disallowSnapshot(String snapshotRoot) throws IOException {
428 | throw new IOException("Invalid operation, do not use proxy");
429 | }
430 |
431 | @Override
432 | public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot, String fromSnapshot, String toSnapshot) throws IOException {
433 | throw new IOException("Invalid operation, do not use proxy");
434 | }
435 |
436 | @Override
437 | public long addCacheDirective(CacheDirectiveInfo directive, EnumSet flags) throws IOException {
438 | return nnProxy.getCacheRegistry().addCacheDirective(directive, flags);
439 | }
440 |
441 | @Override
442 | public void modifyCacheDirective(CacheDirectiveInfo directive, EnumSet flags) throws IOException {
443 | nnProxy.getCacheRegistry().modifyCacheDirective(directive, flags);
444 | }
445 |
446 | @Override
447 | public void removeCacheDirective(long id) throws IOException {
448 | nnProxy.getCacheRegistry().removeCacheDirective(id);
449 | }
450 |
451 | @Override
452 | public BatchedRemoteIterator.BatchedEntries listCacheDirectives(long prevId, CacheDirectiveInfo filter) throws IOException {
453 | return nnProxy.getCacheRegistry().listCacheDirectives(prevId, filter);
454 | }
455 |
456 | @Override
457 | public void addCachePool(CachePoolInfo info) throws IOException {
458 | throw new IOException("Invalid operation, do not use proxy");
459 | }
460 |
461 | @Override
462 | public void modifyCachePool(CachePoolInfo req) throws IOException {
463 | throw new IOException("Invalid operation, do not use proxy");
464 | }
465 |
466 | @Override
467 | public void removeCachePool(String pool) throws IOException {
468 | throw new IOException("Invalid operation, do not use proxy");
469 | }
470 |
471 | @Override
472 | public BatchedRemoteIterator.BatchedEntries listCachePools(String prevPool) throws IOException {
473 | return nnProxy.getCacheRegistry().listCachePools(prevPool);
474 | }
475 |
476 | @Override
477 | public void modifyAclEntries(String src, List aclSpec) throws IOException {
478 | RouteInfo routeInfo = router.route(src);
479 | routeInfo.upstream.modifyAclEntries(routeInfo.realPath, aclSpec);
480 | }
481 |
482 | @Override
483 | public void removeAclEntries(String src, List aclSpec) throws IOException {
484 | RouteInfo routeInfo = router.route(src);
485 | routeInfo.upstream.removeAclEntries(routeInfo.realPath, aclSpec);
486 | }
487 |
488 | @Override
489 | public void removeDefaultAcl(String src) throws IOException {
490 | RouteInfo routeInfo = router.route(src);
491 | routeInfo.upstream.removeDefaultAcl(routeInfo.realPath);
492 | }
493 |
494 | @Override
495 | public void removeAcl(String src) throws IOException {
496 | RouteInfo routeInfo = router.route(src);
497 | routeInfo.upstream.removeAcl(routeInfo.realPath);
498 | }
499 |
500 | @Override
501 | public void setAcl(String src, List aclSpec) throws IOException {
502 | RouteInfo routeInfo = router.route(src);
503 | routeInfo.upstream.setAcl(routeInfo.realPath, aclSpec);
504 | }
505 |
506 | @Override
507 | public AclStatus getAclStatus(String src) throws IOException {
508 | RouteInfo routeInfo = router.route(src);
509 | return routeInfo.upstream.getAclStatus(routeInfo.realPath);
510 | }
511 |
512 | @Override
513 | public void createEncryptionZone(String src, String keyName) throws IOException {
514 | RouteInfo routeInfo = router.route(src);
515 | routeInfo.upstream.createEncryptionZone(routeInfo.realPath, keyName);
516 | }
517 |
518 | @Override
519 | public EncryptionZone getEZForPath(String src) throws IOException {
520 | RouteInfo routeInfo = router.route(src);
521 | return routeInfo.upstream.getEZForPath(routeInfo.realPath);
522 | }
523 |
524 | @Override
525 | public BatchedRemoteIterator.BatchedEntries listEncryptionZones(long prevId) throws IOException {
526 | throw new IOException("Invalid operation, do not use proxy");
527 | }
528 |
529 | @Override
530 | public void setXAttr(String src, XAttr xAttr, EnumSet flag) throws IOException {
531 | RouteInfo routeInfo = router.route(src);
532 | routeInfo.upstream.setXAttr(routeInfo.realPath, xAttr, flag);
533 | }
534 |
535 | @Override
536 | public List getXAttrs(String src, List xAttrs) throws IOException {
537 | RouteInfo routeInfo = router.route(src);
538 | return routeInfo.upstream.getXAttrs(routeInfo.realPath, xAttrs);
539 | }
540 |
541 | @Override
542 | public List listXAttrs(String src) throws IOException {
543 | RouteInfo routeInfo = router.route(src);
544 | return routeInfo.upstream.listXAttrs(routeInfo.realPath);
545 | }
546 |
547 | @Override
548 | public void removeXAttr(String src, XAttr xAttr) throws IOException {
549 | RouteInfo routeInfo = router.route(src);
550 | routeInfo.upstream.removeXAttr(routeInfo.realPath, xAttr);
551 |
552 | }
553 |
554 | @Override
555 | public void checkAccess(String path, FsAction mode) throws IOException {
556 | RouteInfo routeInfo = router.route(path);
557 | routeInfo.upstream.checkAccess(routeInfo.realPath, mode);
558 | }
559 |
560 | @Override
561 | public long getCurrentEditLogTxid() throws IOException {
562 | throw new IOException("Invalid operation, do not use proxy");
563 | }
564 |
565 | @Override
566 | public EventBatchList getEditsFromTxid(long txid) throws IOException {
567 | throw new IOException("Invalid operation, do not use proxy");
568 | }
569 |
570 | }
571 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/server/proxy/ProxyMetrics.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.server.proxy;
2 |
3 | import org.apache.hadoop.classification.InterfaceAudience;
4 | import org.apache.hadoop.classification.InterfaceStability;
5 | import org.apache.hadoop.conf.Configuration;
6 | import org.apache.hadoop.hdfs.DFSConfigKeys;
7 | import org.apache.hadoop.metrics2.MetricsSystem;
8 | import org.apache.hadoop.metrics2.annotation.Metric;
9 | import org.apache.hadoop.metrics2.annotation.Metrics;
10 | import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
11 | import org.apache.hadoop.metrics2.lib.MetricsRegistry;
12 | import org.apache.hadoop.metrics2.lib.MutableCounterLong;
13 | import org.apache.hadoop.metrics2.source.JvmMetrics;
14 | import org.slf4j.Logger;
15 | import org.slf4j.LoggerFactory;
16 |
17 | import static org.apache.hadoop.metrics2.impl.MsInfo.ProcessName;
18 | import static org.apache.hadoop.metrics2.impl.MsInfo.SessionId;
19 |
20 | /** */
21 | @InterfaceAudience.Private
22 | @InterfaceStability.Evolving
23 | @Metrics(name = "ProxyActivity", about = "NameNode proxy metrics", context = "nnproxy")
24 | public class ProxyMetrics {
25 | final MetricsRegistry registry = new MetricsRegistry("nnproxy");
26 |
27 | private static final Logger LOG = LoggerFactory.getLogger(ProxyMetrics.class);
28 |
29 | @Metric
30 | public MutableCounterLong throttledOps;
31 | @Metric
32 | public MutableCounterLong successOps;
33 | @Metric
34 | public MutableCounterLong failedOps;
35 |
36 | JvmMetrics jvmMetrics = null;
37 |
38 | ProxyMetrics(String processName, String sessionId, final JvmMetrics jvmMetrics) {
39 | this.jvmMetrics = jvmMetrics;
40 | registry.tag(ProcessName, processName).tag(SessionId, sessionId);
41 | }
42 |
43 | public static ProxyMetrics create(Configuration conf) {
44 | String sessionId = conf.get(DFSConfigKeys.DFS_METRICS_SESSION_ID_KEY);
45 | String processName = "NNPROXY";
46 | MetricsSystem ms = DefaultMetricsSystem.instance();
47 | JvmMetrics jm = JvmMetrics.create(processName, sessionId, ms);
48 |
49 | return ms.register(new ProxyMetrics(processName, sessionId, jm));
50 | }
51 |
52 | }
53 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/server/proxy/ProxyServer.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.server.proxy;
2 |
3 | import com.bytedance.hadoop.hdfs.server.NNProxy;
4 | import com.bytedance.hadoop.hdfs.ProxyConfig;
5 | import com.google.protobuf.BlockingService;
6 | import org.apache.hadoop.classification.InterfaceAudience;
7 | import org.apache.hadoop.classification.InterfaceStability;
8 | import org.apache.hadoop.conf.Configuration;
9 | import org.apache.hadoop.hdfs.protocol.ClientProtocol;
10 | import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos;
11 | import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
12 | import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB;
13 | import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
14 | import org.apache.hadoop.ipc.ProtobufRpcEngine;
15 | import org.apache.hadoop.ipc.RPC;
16 | import org.slf4j.Logger;
17 | import org.slf4j.LoggerFactory;
18 |
19 | import java.io.IOException;
20 | import java.lang.reflect.InvocationHandler;
21 | import java.lang.reflect.Proxy;
22 | import java.net.InetSocketAddress;
23 |
24 | @InterfaceAudience.Private
25 | @InterfaceStability.Stable
26 | public class ProxyServer {
27 |
28 | private static final Logger LOG = LoggerFactory.getLogger(ProxyServer.class);
29 |
30 | final NNProxy nnProxy;
31 | final Configuration conf;
32 | final InvocationHandler invocationHandler;
33 |
34 | RPC.Server rpcServer;
35 | InetSocketAddress rpcAddress;
36 | ClientProtocol protocol;
37 |
38 | public ProxyServer(NNProxy nnProxy, Configuration conf, InvocationHandler invocationHandler) {
39 | this.nnProxy = nnProxy;
40 | this.conf = conf;
41 | this.invocationHandler = invocationHandler;
42 | }
43 |
44 | public void start() throws IOException {
45 | int rpcHandlerCount = conf.getInt(ProxyConfig.PROXY_HANDLER_COUNT, ProxyConfig.PROXY_HANDLER_COUNT_DEFAULT);
46 | RPC.setProtocolEngine(conf, ClientNamenodeProtocolPB.class,
47 | ProtobufRpcEngine.class);
48 | RPC.setProtocolEngine(conf, NamenodeProtocolPB.class,
49 | ProtobufRpcEngine.class);
50 |
51 | this.protocol = (ClientProtocol) Proxy.newProxyInstance(
52 | this.getClass().getClassLoader(),
53 | new Class[]{ClientProtocol.class},
54 | this.invocationHandler);
55 |
56 | ClientNamenodeProtocolPB proxy = new ClientNamenodeProtocolServerSideTranslatorPB(this.protocol);
57 | BlockingService clientNNPbService = ClientNamenodeProtocolProtos.ClientNamenodeProtocol.
58 | newReflectiveBlockingService(proxy);
59 |
60 | int port = conf.getInt(ProxyConfig.RPC_PORT, ProxyConfig.RPC_PORT_DEFAULT);
61 |
62 | this.rpcServer = new RPC.Builder(conf)
63 | .setProtocol(org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB.class)
64 | .setInstance(clientNNPbService).setBindAddress("0.0.0.0")
65 | .setPort(port).setNumHandlers(rpcHandlerCount)
66 | .setVerbose(false).build();
67 | this.rpcServer.start();
68 |
69 | InetSocketAddress listenAddr = rpcServer.getListenerAddress();
70 | rpcAddress = new InetSocketAddress("0.0.0.0", listenAddr.getPort());
71 | }
72 |
73 | public InetSocketAddress getRpcAddress() {
74 | return rpcAddress;
75 | }
76 |
77 | public ClientProtocol getProtocol() {
78 | return protocol;
79 | }
80 |
81 | public void join() throws InterruptedException {
82 | this.rpcServer.join();
83 | }
84 |
85 | public void shutdown() {
86 | this.rpcServer.stop();
87 | }
88 | }
89 |
90 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/server/proxy/RouteInfo.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.server.proxy;
2 |
3 | import org.apache.hadoop.classification.InterfaceAudience;
4 | import org.apache.hadoop.classification.InterfaceStability;
5 | import org.apache.hadoop.hdfs.protocol.ClientProtocol;
6 |
7 | @InterfaceAudience.Private
8 | @InterfaceStability.Stable
9 | public class RouteInfo {
10 |
11 | final ClientProtocol upstream;
12 | final String realPath;
13 | final String fs;
14 |
15 | public RouteInfo(ClientProtocol upstream, String realPath, String fs) {
16 | this.upstream = upstream;
17 | this.realPath = realPath;
18 | this.fs = fs;
19 | }
20 | }
21 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/server/proxy/Router.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.server.proxy;
2 |
3 | import com.bytedance.hadoop.hdfs.server.NNProxy;
4 | import com.bytedance.hadoop.hdfs.server.exception.WrappedExecutionException;
5 | import com.bytedance.hadoop.hdfs.server.upstream.UpstreamManager;
6 | import org.apache.hadoop.classification.InterfaceAudience;
7 | import org.apache.hadoop.classification.InterfaceStability;
8 | import org.apache.hadoop.conf.Configuration;
9 | import org.apache.hadoop.hdfs.protocol.ClientProtocol;
10 | import org.apache.hadoop.ipc.Server;
11 | import org.slf4j.Logger;
12 | import org.slf4j.LoggerFactory;
13 |
14 | import java.io.IOException;
15 | import java.util.concurrent.ExecutionException;
16 | import java.util.regex.Matcher;
17 | import java.util.regex.Pattern;
18 |
19 | /**
20 | * This routes path or blockPoolId to backend NameNode corresponding to mount table.
21 | */
22 | @InterfaceAudience.Private
23 | @InterfaceStability.Stable
24 | public class Router {
25 |
26 | private static final Logger LOG = LoggerFactory.getLogger(Router.class);
27 |
28 | public static final Pattern TRASH_PATTERN = Pattern.compile("/user/[^/]+/.Trash/[^/]+/(.+)");
29 |
30 | final NNProxy nnProxy;
31 | final Configuration conf;
32 | final UpstreamManager upstreamManager;
33 |
34 | public Router(NNProxy nnProxy, Configuration conf, UpstreamManager upstreamManager) {
35 | this.nnProxy = nnProxy;
36 | this.conf = conf;
37 | this.upstreamManager = upstreamManager;
38 | }
39 |
40 | ClientProtocol getUpstreamProtocol(String user, String fs) throws ExecutionException {
41 | return upstreamManager.getUpstream(user, fs).protocol;
42 | }
43 |
44 | RouteInfo route(String path) throws IOException {
45 | String logicalPath = path;
46 | Matcher mch = TRASH_PATTERN.matcher(path);
47 | if (mch.find()) {
48 | logicalPath = "/" + mch.group(1);
49 | LOG.debug("Hit trash pattern: " + path + " -> " + logicalPath);
50 | }
51 | String fs = nnProxy.getMounts().resolve(logicalPath);
52 | if (fs == null) {
53 | throw new IOException("Not resolved: " + path);
54 | }
55 | if (LOG.isDebugEnabled()) {
56 | LOG.debug("Resolved: " + path + " -> " + fs + path);
57 | }
58 | return new RouteInfo(getProtocol(fs), path, fs);
59 | }
60 |
61 | ClientProtocol getProtocol(String fs) throws IOException {
62 | try {
63 | return getUpstreamProtocol(Server.getRemoteUser().getUserName(), fs);
64 | } catch (ExecutionException e) {
65 | throw new WrappedExecutionException(e.getCause());
66 | }
67 | }
68 |
69 | RouteInfo getRoot() throws IOException {
70 | return route("/");
71 | }
72 |
73 | ClientProtocol getUpstreamForBlockPool(String bpId) throws IOException {
74 | String fs = nnProxy.getBlockPoolRegistry().getFs(bpId);
75 | if (fs == null) {
76 | throw new IOException("Unknown block pool: " + bpId);
77 | }
78 | return getProtocol(fs);
79 | }
80 |
81 | }
82 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/server/proxy/RpcInvocationProxy.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.server.proxy;
2 |
3 | import com.bytedance.hadoop.hdfs.server.NNProxy;
4 | import com.bytedance.hadoop.hdfs.server.exception.WrappedExecutionException;
5 | import com.bytedance.hadoop.hdfs.server.upstream.UpstreamManager;
6 | import org.apache.hadoop.classification.InterfaceAudience;
7 | import org.apache.hadoop.classification.InterfaceStability;
8 | import org.apache.hadoop.conf.Configuration;
9 | import org.apache.hadoop.ipc.*;
10 | import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos;
11 | import org.slf4j.Logger;
12 | import org.slf4j.LoggerFactory;
13 |
14 | import java.lang.reflect.InvocationHandler;
15 | import java.lang.reflect.InvocationTargetException;
16 | import java.lang.reflect.Method;
17 | import java.util.concurrent.atomic.AtomicLong;
18 |
19 | @InterfaceAudience.Private
20 | @InterfaceStability.Stable
21 | public class RpcInvocationProxy implements InvocationHandler {
22 |
23 | private static final Logger LOG = LoggerFactory.getLogger(RpcInvocationProxy.class);
24 |
25 | final NNProxy nnProxy;
26 | final Configuration conf;
27 | final ProxyClientProtocolHandler protocolHandler;
28 | volatile boolean isShuttingDown;
29 | final AtomicLong activeRequests;
30 |
31 | public RpcInvocationProxy(NNProxy nnProxy, Configuration conf, UpstreamManager upstreamManager) {
32 | this.nnProxy = nnProxy;
33 | this.conf = conf;
34 | this.protocolHandler = new ProxyClientProtocolHandler(nnProxy, conf, upstreamManager);
35 | this.isShuttingDown = false;
36 | this.activeRequests = new AtomicLong(0);
37 | }
38 |
39 | void setupClientAddress() {
40 | String clientAddress = Server.getRemoteAddress();
41 | Client.setClientAddress(clientAddress);
42 | }
43 |
44 | @Override
45 | public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
46 | if (isShuttingDown) {
47 | throw new StandbyException("Proxy is shutting down");
48 | }
49 | try {
50 | activeRequests.incrementAndGet();
51 | setupClientAddress();
52 | return method.invoke(protocolHandler, args);
53 | } catch (InvocationTargetException e) {
54 | LOG.error("Error handling client", e);
55 | if (e.getCause() instanceof RemoteException) {
56 | // needs to pass RemoteException to client untouched
57 | RemoteException remoteException = (RemoteException) e.getCause();
58 | throw new ProxyRpcServerException(
59 | RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto.ERROR,
60 | remoteException.getErrorCode(),
61 | remoteException.getClassName(),
62 | remoteException.getMessage());
63 | } else {
64 | throw e.getCause();
65 | }
66 | } catch (WrappedExecutionException e) {
67 | LOG.error("Error handling client", e);
68 | throw e.getCause();
69 | } catch (Exception e) {
70 | // log errors here otherwise no trace is left on server side
71 | LOG.error("Error handling client", e);
72 | throw e;
73 | } finally {
74 | activeRequests.decrementAndGet();
75 | }
76 | }
77 |
78 | public void shutdown() {
79 | isShuttingDown = true;
80 | // sleep a moment just to make sure all requests are accounted in activeRequests
81 | try {
82 | Thread.sleep(1000);
83 | } catch (InterruptedException e) {
84 |
85 | }
86 | while (activeRequests.get() > 0) {
87 | try {
88 | Thread.sleep(1000);
89 | } catch (InterruptedException e) {
90 |
91 | }
92 | LOG.info("Waiting for all requests to finish... " + activeRequests.get() + " left");
93 | }
94 | }
95 | }
96 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/server/quota/ThrottleInvocationHandler.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.server.quota;
2 |
3 | import com.bytedance.hadoop.hdfs.server.NNProxy;
4 | import com.google.common.base.Function;
5 | import com.google.common.base.Preconditions;
6 | import org.apache.hadoop.classification.InterfaceAudience;
7 | import org.apache.hadoop.classification.InterfaceStability;
8 | import org.apache.hadoop.ipc.StandbyException;
9 |
10 | import java.lang.reflect.InvocationHandler;
11 | import java.lang.reflect.InvocationTargetException;
12 | import java.lang.reflect.Method;
13 | import java.util.concurrent.atomic.AtomicLong;
14 |
15 | @InterfaceAudience.Private
16 | @InterfaceStability.Stable
17 | public class ThrottleInvocationHandler implements InvocationHandler {
18 |
19 | final Object underlying;
20 | final Function opCounter;
21 | final long threshold;
22 |
23 | public ThrottleInvocationHandler(Object underlying, Function opCounter, long threshold) {
24 | this.underlying = underlying;
25 | this.opCounter = opCounter;
26 | this.threshold = threshold;
27 | }
28 |
29 | @Override
30 | public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
31 | AtomicLong counter = opCounter.apply(method);
32 | Preconditions.checkState(counter != null);
33 | long current = counter.getAndIncrement();
34 | try {
35 | if (current > threshold) {
36 | NNProxy.proxyMetrics.throttledOps.incr();
37 | throw new StandbyException("Too many requests (" + current + "/" + threshold + "), try later");
38 | }
39 | Object ret = method.invoke(underlying, args);
40 | NNProxy.proxyMetrics.successOps.incr();
41 | return ret;
42 | } catch (InvocationTargetException e) {
43 | NNProxy.proxyMetrics.failedOps.incr();
44 | throw e.getCause();
45 | } finally {
46 | counter.decrementAndGet();
47 | }
48 | }
49 | }
50 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/server/upstream/UpstreamManager.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.server.upstream;
2 |
3 | import com.bytedance.hadoop.hdfs.ProxyConfig;
4 | import com.bytedance.hadoop.hdfs.server.NNProxy;
5 | import com.bytedance.hadoop.hdfs.server.quota.ThrottleInvocationHandler;
6 | import com.google.common.base.Function;
7 | import com.google.common.cache.CacheBuilder;
8 | import com.google.common.cache.CacheLoader;
9 | import com.google.common.cache.LoadingCache;
10 | import org.apache.hadoop.classification.InterfaceAudience;
11 | import org.apache.hadoop.classification.InterfaceStability;
12 | import org.apache.hadoop.conf.Configuration;
13 | import org.apache.hadoop.hdfs.NameNodeProxies;
14 | import org.apache.hadoop.hdfs.protocol.ClientProtocol;
15 | import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
16 | import org.apache.hadoop.security.SaslRpcServer;
17 | import org.apache.hadoop.security.UserGroupInformation;
18 | import org.slf4j.Logger;
19 | import org.slf4j.LoggerFactory;
20 |
21 | import java.io.IOException;
22 | import java.lang.reflect.Method;
23 | import java.lang.reflect.Proxy;
24 | import java.net.URI;
25 | import java.util.Map;
26 | import java.util.concurrent.ConcurrentHashMap;
27 | import java.util.concurrent.ExecutionException;
28 | import java.util.concurrent.TimeUnit;
29 | import java.util.concurrent.atomic.AtomicLong;
30 |
31 | @InterfaceAudience.Private
32 | @InterfaceStability.Stable
33 | public class UpstreamManager {
34 |
35 | private static final Logger LOG = LoggerFactory.getLogger(UpstreamManager.class);
36 |
37 | final NNProxy nnProxy;
38 | final Configuration conf;
39 | final LoadingCache upstreamCache;
40 | final long maxConrruentRequestPerFs;
41 | final Map fsRequests;
42 |
43 | public static class Upstream {
44 | public final ClientProtocol protocol;
45 | public final NameNodeProxies.ProxyAndInfo proxyAndInfo;
46 | public final NameNodeProxies.ProxyAndInfo nnProxyAndInfo;
47 | public volatile int fsIndex;
48 |
49 | public Upstream(ClientProtocol protocol,
50 | NameNodeProxies.ProxyAndInfo proxyAndInfo,
51 | NameNodeProxies.ProxyAndInfo nnProxyAndInfo) {
52 | this.protocol = protocol;
53 | this.proxyAndInfo = proxyAndInfo;
54 | this.nnProxyAndInfo = nnProxyAndInfo;
55 | }
56 |
57 | public int getFsIndex() {
58 | return fsIndex;
59 | }
60 |
61 | public void setFsIndex(int fsIndex) {
62 | this.fsIndex = fsIndex;
63 | }
64 | }
65 |
66 | public static class UpstreamTicket {
67 | public final String user;
68 | public final String fs;
69 |
70 | public UpstreamTicket(String user, String fs) {
71 | this.user = user;
72 | this.fs = fs;
73 | }
74 |
75 | @Override
76 | public boolean equals(Object o) {
77 | if (this == o) return true;
78 | if (!(o instanceof UpstreamTicket)) return false;
79 |
80 | UpstreamTicket that = (UpstreamTicket) o;
81 |
82 | if (user != null ? !user.equals(that.user) : that.user != null) return false;
83 | return !(fs != null ? !fs.equals(that.fs) : that.fs != null);
84 | }
85 |
86 | @Override
87 | public int hashCode() {
88 | int result = user != null ? user.hashCode() : 0;
89 | result = 31 * result + (fs != null ? fs.hashCode() : 0);
90 | return result;
91 | }
92 | }
93 |
94 | public UpstreamManager(NNProxy nnProxy, Configuration conf) {
95 | this.nnProxy = nnProxy;
96 | this.conf = conf;
97 | final long cacheExpire =
98 | conf.getLong(ProxyConfig.USER_PROXY_EXPIRE_MS, ProxyConfig.USER_PROXY_EXPIRE_MS_DEFAULT);
99 | maxConrruentRequestPerFs =
100 | conf.getLong(ProxyConfig.MAX_CONCURRENT_REQUEST_PER_FS, ProxyConfig.MAX_CONCURRENT_REQUEST_PER_FS_DEFAULT);
101 | this.upstreamCache = CacheBuilder.newBuilder()
102 | .expireAfterAccess(cacheExpire, TimeUnit.MILLISECONDS)
103 | .build(new CacheLoader() {
104 | @Override
105 | public Upstream load(UpstreamTicket ticket) throws Exception {
106 | return makeUpstream(ticket);
107 | }
108 | });
109 | this.fsRequests = new ConcurrentHashMap<>();
110 | }
111 |
112 | synchronized T wrapWithThrottle(final String key, final T underlying, final Class xface) {
113 | if (!fsRequests.containsKey(key)) {
114 | fsRequests.put(key, new AtomicLong(0L));
115 | }
116 | final Function counterGetter = new Function() {
117 | @Override
118 | public AtomicLong apply(Method method) {
119 | return fsRequests.get(key);
120 | }
121 | };
122 | ThrottleInvocationHandler throttleHandler = new ThrottleInvocationHandler(underlying, counterGetter, maxConrruentRequestPerFs);
123 | return (T) Proxy.newProxyInstance(this.getClass().getClassLoader(),
124 | new Class[]{xface}, throttleHandler);
125 | }
126 |
127 | synchronized Upstream makeUpstream(UpstreamTicket ticket) throws IOException {
128 | if (ticket.user != null) {
129 | UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser(ticket.user,
130 | SaslRpcServer.AuthMethod.SIMPLE));
131 | } else {
132 | UserGroupInformation.setLoginUser(null);
133 | }
134 | URI fsUri = URI.create(ticket.fs);
135 | NameNodeProxies.ProxyAndInfo proxyAndInfo = NameNodeProxies.createProxy(conf, fsUri, ClientProtocol.class);
136 | NameNodeProxies.ProxyAndInfo nnProxyAndInfo = NameNodeProxies.createProxy(conf, fsUri, NamenodeProtocol.class);
137 | LOG.info("New upstream: " + ticket.user + "@" + ticket.fs);
138 | ClientProtocol clientProtocol = (ClientProtocol) proxyAndInfo.getProxy();
139 | return new Upstream(wrapWithThrottle(ticket.fs, clientProtocol, ClientProtocol.class), proxyAndInfo, nnProxyAndInfo);
140 | }
141 |
142 | public Upstream getUpstream(String user, String fs) throws ExecutionException {
143 | return upstreamCache.get(new UpstreamTicket(user, fs));
144 | }
145 | }
146 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/tools/DumpMount.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.tools;
2 |
3 | import com.bytedance.hadoop.hdfs.server.mount.MountsManager;
4 | import org.apache.hadoop.conf.Configuration;
5 | import org.apache.hadoop.hdfs.HdfsConfiguration;
6 | import org.apache.hadoop.util.Tool;
7 | import org.apache.hadoop.util.ToolRunner;
8 | import org.slf4j.Logger;
9 | import org.slf4j.LoggerFactory;
10 |
11 | /** */
12 | public class DumpMount implements Tool {
13 |
14 | private static final Logger LOG = LoggerFactory.getLogger(DumpMount.class);
15 |
16 | Configuration conf;
17 |
18 | public static void main(String[] args) throws Exception {
19 | DumpMount main = new DumpMount();
20 | System.exit(ToolRunner.run(new HdfsConfiguration(), main, args));
21 | }
22 |
23 | @Override
24 | public int run(String[] args) throws Exception {
25 | MountsManager mountsManager = new MountsManager();
26 | mountsManager.init(conf);
27 | mountsManager.start();
28 | mountsManager.waitUntilInstalled();
29 | System.out.println(mountsManager.dump());
30 | return 0;
31 | }
32 |
33 | @Override
34 | public void setConf(Configuration conf) {
35 | this.conf = conf;
36 | }
37 |
38 | @Override
39 | public Configuration getConf() {
40 | return conf;
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/java/com/bytedance/hadoop/hdfs/tools/LoadMount.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.tools;
2 |
3 | import com.bytedance.hadoop.hdfs.server.mount.MountsManager;
4 | import org.apache.commons.io.IOUtils;
5 | import org.apache.hadoop.conf.Configuration;
6 | import org.apache.hadoop.hdfs.HdfsConfiguration;
7 | import org.apache.hadoop.util.Tool;
8 | import org.apache.hadoop.util.ToolRunner;
9 | import org.slf4j.Logger;
10 | import org.slf4j.LoggerFactory;
11 |
12 | /** */
13 | public class LoadMount implements Tool {
14 |
15 | private static final Logger LOG = LoggerFactory.getLogger(LoadMount.class);
16 |
17 | Configuration conf;
18 |
19 | public static void main(String[] args) throws Exception {
20 | LoadMount main = new LoadMount();
21 | System.exit(ToolRunner.run(new HdfsConfiguration(), main, args));
22 | }
23 |
24 | @Override
25 | public int run(String[] args) throws Exception {
26 | String mounts = IOUtils.toString(System.in);
27 | MountsManager mountsManager = new MountsManager();
28 | mountsManager.init(conf);
29 | mountsManager.start();
30 | mountsManager.load(mounts);
31 | return 0;
32 | }
33 |
34 | @Override
35 | public void setConf(Configuration conf) {
36 | this.conf = conf;
37 | }
38 |
39 | @Override
40 | public Configuration getConf() {
41 | return conf;
42 | }
43 | }
44 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/main/resources/bdversion.properties:
--------------------------------------------------------------------------------
1 | gitrev=${buildNumber}
2 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/test/java/com/bytedance/hadoop/hdfs/server/MiniNNProxy.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.server;
2 |
3 | import com.bytedance.hadoop.hdfs.server.NNProxy;
4 | import com.bytedance.hadoop.hdfs.server.mount.MountsManager;
5 | import org.apache.hadoop.conf.Configuration;
6 | import org.apache.hadoop.fs.FileSystem;
7 | import org.apache.hadoop.hdfs.DFSClient;
8 | import org.apache.hadoop.hdfs.MiniDFSCluster;
9 | import org.apache.hadoop.hdfs.NameNodeProxies;
10 | import org.apache.hadoop.hdfs.protocol.ClientProtocol;
11 | import org.apache.hadoop.security.UserGroupInformation;
12 |
13 | import java.net.URI;
14 |
15 | /** */
16 | public class MiniNNProxy extends NNProxy {
17 |
18 | final String mountTable;
19 | final MiniDFSCluster[] clusters;
20 | ClientProtocol clientProtocol;
21 | DFSClient dfs;
22 | FileSystem fs;
23 |
24 | class MockedMountsManager extends MountsManager {
25 |
26 | @Override
27 | protected void serviceInit(Configuration conf) throws Exception {
28 | handleMountTableChange(mountTable.getBytes());
29 | }
30 |
31 | @Override
32 | protected void serviceStart() throws Exception {
33 | }
34 |
35 | @Override
36 | protected void serviceStop() throws Exception {
37 | }
38 | }
39 |
40 | public MiniNNProxy(Configuration conf, String mountTable, MiniDFSCluster[] clusters) throws Exception {
41 | super(conf);
42 | this.mountTable = mountTable;
43 | this.clusters = clusters;
44 | this.mounts = new MockedMountsManager();
45 | this.start();
46 |
47 |
48 | UserGroupInformation curr = UserGroupInformation.getCurrentUser();
49 | clientProtocol = NameNodeProxies.createNonHAProxy(conf,
50 | getRpcAddress(), ClientProtocol.class,
51 | curr, false).getProxy();
52 | dfs = new DFSClient(URI.create("hdfs://127.0.0.1:" + getRpcAddress().getPort()), conf);
53 | fs = FileSystem.newInstance(URI.create("hdfs://127.0.0.1:" + getRpcAddress().getPort()), conf, curr.getUserName());
54 | }
55 |
56 | public ClientProtocol getClientProtocol() {
57 | return clientProtocol;
58 | }
59 |
60 | public DFSClient getDfs() {
61 | return dfs;
62 | }
63 |
64 | public FileSystem getFs() {
65 | return fs;
66 | }
67 | }
68 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/test/java/com/bytedance/hadoop/hdfs/server/TestNNProxy.java:
--------------------------------------------------------------------------------
1 | package com.bytedance.hadoop.hdfs.server;
2 |
3 | import com.bytedance.hadoop.hdfs.ProxyConfig;
4 | import org.apache.hadoop.conf.Configuration;
5 | import org.apache.hadoop.fs.FileSystem;
6 | import org.apache.hadoop.fs.Path;
7 | import org.apache.hadoop.hdfs.DFSClient;
8 | import org.apache.hadoop.hdfs.MiniDFSCluster;
9 | import org.apache.hadoop.hdfs.protocol.ClientProtocol;
10 | import org.junit.Before;
11 | import org.junit.Test;
12 | import org.slf4j.Logger;
13 | import org.slf4j.LoggerFactory;
14 |
15 | import java.io.IOException;
16 |
17 | import static org.junit.Assert.assertEquals;
18 |
19 | /** */
20 | public class TestNNProxy {
21 |
22 | private static final Logger LOG = LoggerFactory.getLogger(TestNNProxy.class);
23 |
24 | Configuration conf;
25 | MiniNNProxy nnProxy;
26 | ClientProtocol clientProtocol;
27 | DFSClient dfs;
28 | FileSystem fs;
29 |
30 | @Before
31 | public void setUp() throws Exception {
32 | conf = new Configuration();
33 | conf.setInt(ProxyConfig.PROXY_HANDLER_COUNT, 12);
34 | MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
35 | cluster.waitActive();
36 | int port = cluster.getNameNode().getNameNodeAddress().getPort();
37 | nnProxy = new MiniNNProxy(conf, "hdfs://127.0.0.1:" + port + " /", new MiniDFSCluster[]{cluster});
38 | clientProtocol = nnProxy.getClientProtocol();
39 | dfs = nnProxy.getDfs();
40 | fs = nnProxy.getFs();
41 | }
42 |
43 | @Test
44 | public void testGetListing() throws IOException {
45 | fs.create(new Path("/test"), (short) 3).close();
46 | assertEquals(1, fs.listStatus(new Path("/")).length);
47 | assertEquals("test", fs.listStatus(new Path("/"))[0].getPath().getName());
48 | }
49 |
50 |
51 | }
52 |
--------------------------------------------------------------------------------
/hadoop-nnproxy/src/test/java/org/apache/hadoop/hdfs/TestProxyFs.java:
--------------------------------------------------------------------------------
1 | package org.apache.hadoop.hdfs;
2 |
3 | import com.bytedance.hadoop.hdfs.server.MiniNNProxy;
4 | import org.apache.hadoop.conf.Configuration;
5 | import org.apache.hadoop.fs.*;
6 | import org.apache.hadoop.fs.permission.FsPermission;
7 | import org.apache.hadoop.hdfs.protocol.ClientProtocol;
8 | import org.apache.hadoop.security.UserGroupInformation;
9 | import org.apache.hadoop.util.Time;
10 | import org.junit.Before;
11 | import org.junit.Test;
12 | import org.slf4j.Logger;
13 | import org.slf4j.LoggerFactory;
14 |
15 | import java.io.FileNotFoundException;
16 | import java.util.Random;
17 |
18 | import static org.junit.Assert.*;
19 |
20 | /** */
21 | public class TestProxyFs {
22 |
23 | private static final Random RAN = new Random();
24 | private static final Logger LOG = LoggerFactory.getLogger(TestProxyFs.class);
25 |
26 | Configuration conf;
27 | MiniNNProxy nnProxy;
28 | ClientProtocol clientProtocol;
29 | DFSClient dfs;
30 | FileSystem fs;
31 | MiniDFSCluster cluster;
32 |
33 | @Before
34 | public void setUp() throws Exception {
35 | conf = new Configuration();
36 | conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
37 | conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
38 | conf.setInt("dfs.namenode.fs-limits.min-block-size", 1024);
39 | cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
40 | cluster.waitActive();
41 | int port = cluster.getNameNode().getNameNodeAddress().getPort();
42 | nnProxy = new MiniNNProxy(conf, "hdfs://127.0.0.1:" + port + " /", new MiniDFSCluster[]{cluster});
43 | clientProtocol = nnProxy.getClientProtocol();
44 | dfs = nnProxy.getDfs();
45 | fs = nnProxy.getFs();
46 | }
47 |
48 | @Test
49 | public void testDFSClient() throws Exception {
50 | final long grace = 1000L;
51 | try {
52 | final String filepathstring = "/test/LeaseChecker/foo";
53 | final Path[] filepaths = new Path[4];
54 | for (int i = 0; i < filepaths.length; i++) {
55 | filepaths[i] = new Path(filepathstring + i);
56 | }
57 | final long millis = Time.now();
58 |
59 | {
60 | final DistributedFileSystem dfs = cluster.getFileSystem();
61 | dfs.dfs.getLeaseRenewer().setGraceSleepPeriod(grace);
62 | assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
63 |
64 | {
65 | //create a file
66 | final FSDataOutputStream out = dfs.create(filepaths[0]);
67 | assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
68 | //write something
69 | out.writeLong(millis);
70 | assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
71 | //close
72 | out.close();
73 | Thread.sleep(grace / 4 * 3);
74 | //within grace period
75 | assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
76 | for (int i = 0; i < 3; i++) {
77 | if (dfs.dfs.getLeaseRenewer().isRunning()) {
78 | Thread.sleep(grace / 2);
79 | }
80 | }
81 | //passed grace period
82 | assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
83 | }
84 |
85 | {
86 | //create file1
87 | final FSDataOutputStream out1 = dfs.create(filepaths[1]);
88 | assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
89 | //create file2
90 | final FSDataOutputStream out2 = dfs.create(filepaths[2]);
91 | assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
92 |
93 | //write something to file1
94 | out1.writeLong(millis);
95 | assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
96 | //close file1
97 | out1.close();
98 | assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
99 |
100 | //write something to file2
101 | out2.writeLong(millis);
102 | assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
103 | //close file2
104 | out2.close();
105 | Thread.sleep(grace / 4 * 3);
106 | //within grace period
107 | assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
108 | }
109 |
110 | {
111 | //create file3
112 | final FSDataOutputStream out3 = dfs.create(filepaths[3]);
113 | assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
114 | Thread.sleep(grace / 4 * 3);
115 | //passed previous grace period, should still running
116 | assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
117 | //write something to file3
118 | out3.writeLong(millis);
119 | assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
120 | //close file3
121 | out3.close();
122 | assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
123 | Thread.sleep(grace / 4 * 3);
124 | //within grace period
125 | assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
126 | for (int i = 0; i < 3; i++) {
127 | if (dfs.dfs.getLeaseRenewer().isRunning()) {
128 | Thread.sleep(grace / 2);
129 | }
130 | }
131 | //passed grace period
132 | assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
133 | }
134 |
135 | dfs.close();
136 | }
137 |
138 | {
139 | // Check to see if opening a non-existent file triggers a FNF
140 | FileSystem fs = cluster.getFileSystem();
141 | Path dir = new Path("/wrwelkj");
142 | assertFalse("File should not exist for test.", fs.exists(dir));
143 |
144 | try {
145 | FSDataInputStream in = fs.open(dir);
146 | try {
147 | in.close();
148 | fs.close();
149 | } finally {
150 | assertTrue("Did not get a FileNotFoundException for non-existing" +
151 | " file.", false);
152 | }
153 | } catch (FileNotFoundException fnf) {
154 | // This is the proper exception to catch; move on.
155 | }
156 |
157 | }
158 |
159 | {
160 | final DistributedFileSystem dfs = cluster.getFileSystem();
161 | assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
162 |
163 | //open and check the file
164 | FSDataInputStream in = dfs.open(filepaths[0]);
165 | assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
166 | assertEquals(millis, in.readLong());
167 | assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
168 | in.close();
169 | assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
170 | dfs.close();
171 | }
172 |
173 | { // test accessing DFS with ip address. should work with any hostname
174 | // alias or ip address that points to the interface that NameNode
175 | // is listening on. In this case, it is localhost.
176 | String uri = "hdfs://127.0.0.1:" + cluster.getNameNodePort() +
177 | "/test/ipAddress/file";
178 | Path path = new Path(uri);
179 | FileSystem fs = FileSystem.get(path.toUri(), conf);
180 | FSDataOutputStream out = fs.create(path);
181 | byte[] buf = new byte[1024];
182 | out.write(buf);
183 | out.close();
184 |
185 | FSDataInputStream in = fs.open(path);
186 | in.readFully(buf);
187 | in.close();
188 | fs.close();
189 | }
190 | } finally {
191 | if (cluster != null) {
192 | cluster.shutdown();
193 | }
194 | }
195 | }
196 |
197 | @Test
198 | public void testFileChecksum() throws Exception {
199 | final long seed = RAN.nextLong();
200 | System.out.println("seed=" + seed);
201 | RAN.setSeed(seed);
202 |
203 | final FileSystem hdfs = fs;
204 |
205 | final UserGroupInformation current = UserGroupInformation.getCurrentUser();
206 | final UserGroupInformation ugi = UserGroupInformation.createUserForTesting(
207 | current.getShortUserName() + "x", new String[]{"user"});
208 |
209 | try {
210 | hdfs.getFileChecksum(new Path(
211 | "/test/TestNonExistingFile"));
212 | fail("Expecting FileNotFoundException");
213 | } catch (FileNotFoundException e) {
214 | assertTrue("Not throwing the intended exception message", e.getMessage()
215 | .contains("File does not exist: /test/TestNonExistingFile"));
216 | }
217 |
218 | try {
219 | Path path = new Path("/test/TestExistingDir/");
220 | hdfs.mkdirs(path);
221 | hdfs.getFileChecksum(path);
222 | fail("Expecting FileNotFoundException");
223 | } catch (FileNotFoundException e) {
224 | assertTrue("Not throwing the intended exception message", e.getMessage()
225 | .contains("Path is not a file: /test/TestExistingDir"));
226 | }
227 |
228 | final Path dir = new Path("/filechecksum");
229 | final int block_size = 1024;
230 | final int buffer_size = conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096);
231 | conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 512);
232 |
233 | //try different number of blocks
234 | for (int n = 0; n < 5; n++) {
235 | //generate random data
236 | final byte[] data = new byte[RAN.nextInt(block_size / 2 - 1) + n * block_size + 1];
237 | RAN.nextBytes(data);
238 | System.out.println("data.length=" + data.length);
239 |
240 | //write data to a file
241 | final Path foo = new Path(dir, "foo" + n);
242 | {
243 | final FSDataOutputStream out = hdfs.create(foo, false, buffer_size,
244 | (short) 2, block_size);
245 | out.write(data);
246 | out.close();
247 | }
248 |
249 | //compute checksum
250 | final FileChecksum hdfsfoocs = hdfs.getFileChecksum(foo);
251 | System.out.println("hdfsfoocs=" + hdfsfoocs);
252 |
253 | //create a zero byte file
254 | final Path zeroByteFile = new Path(dir, "zeroByteFile" + n);
255 | {
256 | final FSDataOutputStream out = hdfs.create(zeroByteFile, false, buffer_size,
257 | (short) 2, block_size);
258 | out.close();
259 | }
260 |
261 | // verify the magic val for zero byte files
262 | {
263 | final FileChecksum zeroChecksum = hdfs.getFileChecksum(zeroByteFile);
264 | assertEquals(zeroChecksum.toString(),
265 | "MD5-of-0MD5-of-0CRC32:70bc8f4b72a86921468bf8e8441dce51");
266 | }
267 |
268 | //write another file
269 | final Path bar = new Path(dir, "bar" + n);
270 | {
271 | final FSDataOutputStream out = hdfs.create(bar, false, buffer_size,
272 | (short) 2, block_size);
273 | out.write(data);
274 | out.close();
275 | }
276 |
277 | { //verify checksum
278 | final FileChecksum barcs = hdfs.getFileChecksum(bar);
279 | final int barhashcode = barcs.hashCode();
280 | assertEquals(hdfsfoocs.hashCode(), barhashcode);
281 | assertEquals(hdfsfoocs, barcs);
282 |
283 | }
284 |
285 | hdfs.setPermission(dir, new FsPermission((short) 0));
286 | hdfs.setPermission(dir, new FsPermission((short) 0777));
287 | }
288 | cluster.shutdown();
289 | }
290 |
291 |
292 | }
293 |
--------------------------------------------------------------------------------
/hadoop-patches/0001-Add-clientAddress-to-rpc-header-For-proxy-usage.patch:
--------------------------------------------------------------------------------
1 | From 57936bf50d7a8b289a958fcc4bf6b2522203b644 Mon Sep 17 00:00:00 2001
2 | From: hetianyi
3 | Date: Thu, 10 Mar 2016 18:36:06 +0800
4 | Subject: [PATCH] Add clientAddress to rpc header (For proxy usage)
5 |
6 | ---
7 | .../main/java/org/apache/hadoop/ipc/Client.java | 12 ++++++---
8 | .../main/java/org/apache/hadoop/ipc/Server.java | 29 +++++++++++++++++++---
9 | .../org/apache/hadoop/security/SaslRpcClient.java | 2 +-
10 | .../java/org/apache/hadoop/util/ProtoUtil.java | 5 +++-
11 | .../hadoop-common/src/main/proto/RpcHeader.proto | 1 +
12 | .../java/org/apache/hadoop/util/TestProtoUtil.java | 2 +-
13 | 6 files changed, 41 insertions(+), 10 deletions(-)
14 |
15 | diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
16 | index d9385fb..51bb2f2 100644
17 | --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
18 | +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
19 | @@ -110,7 +110,7 @@
20 |
21 | private static final ThreadLocal callId = new ThreadLocal();
22 | private static final ThreadLocal retryCount = new ThreadLocal();
23 | -
24 | + private static final ThreadLocal clientAddress = new ThreadLocal();
25 | /** Set call id and retry count for the next call. */
26 | public static void setCallIdAndRetryCount(int cid, int rc) {
27 | Preconditions.checkArgument(cid != RpcConstants.INVALID_CALL_ID);
28 | @@ -121,6 +121,10 @@ public static void setCallIdAndRetryCount(int cid, int rc) {
29 | retryCount.set(rc);
30 | }
31 |
32 | + public static void setClientAddress(String s) {
33 | + clientAddress.set(s);
34 | + }
35 | +
36 | private Hashtable connections =
37 | new Hashtable();
38 |
39 | @@ -418,7 +422,7 @@ public Connection(ConnectionId remoteId, int serviceClass) throws IOException {
40 | RpcRequestHeaderProto pingHeader = ProtoUtil
41 | .makeRpcRequestHeader(RpcKind.RPC_PROTOCOL_BUFFER,
42 | OperationProto.RPC_FINAL_PACKET, PING_CALL_ID,
43 | - RpcConstants.INVALID_RETRY_COUNT, clientId);
44 | + RpcConstants.INVALID_RETRY_COUNT, clientId, clientAddress.get());
45 | pingHeader.writeDelimitedTo(pingRequest);
46 | }
47 | this.pingInterval = remoteId.getPingInterval();
48 | @@ -896,7 +900,7 @@ private void writeConnectionContext(ConnectionId remoteId,
49 | RpcRequestHeaderProto connectionContextHeader = ProtoUtil
50 | .makeRpcRequestHeader(RpcKind.RPC_PROTOCOL_BUFFER,
51 | OperationProto.RPC_FINAL_PACKET, CONNECTION_CONTEXT_CALL_ID,
52 | - RpcConstants.INVALID_RETRY_COUNT, clientId);
53 | + RpcConstants.INVALID_RETRY_COUNT, clientId, clientAddress.get());
54 | RpcRequestMessageWrapper request =
55 | new RpcRequestMessageWrapper(connectionContextHeader, message);
56 |
57 | @@ -1006,7 +1010,7 @@ public void sendRpcRequest(final Call call)
58 | final DataOutputBuffer d = new DataOutputBuffer();
59 | RpcRequestHeaderProto header = ProtoUtil.makeRpcRequestHeader(
60 | call.rpcKind, OperationProto.RPC_FINAL_PACKET, call.id, call.retry,
61 | - clientId);
62 | + clientId, clientAddress.get());
63 | header.writeDelimitedTo(d);
64 | call.rpcRequest.write(d);
65 |
66 | diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
67 | index 16cb232..43e8ed4 100644
68 | --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
69 | +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
70 | @@ -315,7 +315,13 @@ public static int getCallRetryCount() {
71 | */
72 | public static InetAddress getRemoteIp() {
73 | Call call = CurCall.get();
74 | - return (call != null && call.connection != null) ? call.connection
75 | + if (call == null) {
76 | + return null;
77 | + }
78 | + if (call.clientAddress != null) {
79 | + return call.clientAddress;
80 | + }
81 | + return (call.connection != null) ? call.connection
82 | .getHostInetAddress() : null;
83 | }
84 |
85 | @@ -510,6 +516,8 @@ public synchronized void refreshCallQueue(Configuration conf) {
86 | private final byte[] clientId;
87 | private final Span traceSpan; // the tracing span on the server side
88 |
89 | + private final InetAddress clientAddress;
90 | +
91 | public Call(int id, int retryCount, Writable param,
92 | Connection connection) {
93 | this(id, retryCount, param, connection, RPC.RpcKind.RPC_BUILTIN,
94 | @@ -522,7 +530,12 @@ public Call(int id, int retryCount, Writable param, Connection connection,
95 | }
96 |
97 | public Call(int id, int retryCount, Writable param, Connection connection,
98 | - RPC.RpcKind kind, byte[] clientId, Span span) {
99 | + RPC.RpcKind kind, byte[] clientId, Span span) {
100 | + this(id, retryCount, param, connection, kind, clientId, span, null);
101 | + }
102 | +
103 | + public Call(int id, int retryCount, Writable param, Connection connection,
104 | + RPC.RpcKind kind, byte[] clientId, Span span, InetAddress clientAddress) {
105 | this.callId = id;
106 | this.retryCount = retryCount;
107 | this.rpcRequest = param;
108 | @@ -532,6 +545,7 @@ public Call(int id, int retryCount, Writable param, Connection connection,
109 | this.rpcKind = kind;
110 | this.clientId = clientId;
111 | this.traceSpan = span;
112 | + this.clientAddress = clientAddress;
113 | }
114 |
115 | @Override
116 | @@ -1861,9 +1875,18 @@ private void processRpcRequest(RpcRequestHeaderProto header,
117 | traceSpan = Trace.startSpan(rpcRequest.toString(), parentSpan).detach();
118 | }
119 |
120 | + InetAddress clientAddress = null;
121 | + if (header.hasClientAddress()) {
122 | + try {
123 | + clientAddress = InetAddress.getByName(header.getClientAddress());
124 | + } catch (UnknownHostException e) {
125 | + LOG.warn("Invalid client address:" + header.getClientAddress());
126 | + clientAddress = null;
127 | + }
128 | + }
129 | Call call = new Call(header.getCallId(), header.getRetryCount(),
130 | rpcRequest, this, ProtoUtil.convert(header.getRpcKind()),
131 | - header.getClientId().toByteArray(), traceSpan);
132 | + header.getClientId().toByteArray(), traceSpan, clientAddress);
133 |
134 | callQueue.put(call); // queue the call; maybe blocked here
135 | incRpcCount(); // Increment the rpc count
136 | diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java
137 | index dfb0898..0becc44 100644
138 | --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java
139 | +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java
140 | @@ -95,7 +95,7 @@
141 | private static final RpcRequestHeaderProto saslHeader = ProtoUtil
142 | .makeRpcRequestHeader(RpcKind.RPC_PROTOCOL_BUFFER,
143 | OperationProto.RPC_FINAL_PACKET, AuthProtocol.SASL.callId,
144 | - RpcConstants.INVALID_RETRY_COUNT, RpcConstants.DUMMY_CLIENT_ID);
145 | + RpcConstants.INVALID_RETRY_COUNT, RpcConstants.DUMMY_CLIENT_ID, null);
146 | private static final RpcSaslProto negotiateRequest =
147 | RpcSaslProto.newBuilder().setState(SaslState.NEGOTIATE).build();
148 |
149 | diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java
150 | index 36b5ff1..d40c2c9 100644
151 | --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java
152 | +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java
153 | @@ -163,10 +163,13 @@ static RpcKindProto convert(RPC.RpcKind kind) {
154 |
155 | public static RpcRequestHeaderProto makeRpcRequestHeader(RPC.RpcKind rpcKind,
156 | RpcRequestHeaderProto.OperationProto operation, int callId,
157 | - int retryCount, byte[] uuid) {
158 | + int retryCount, byte[] uuid, String clientAddress) {
159 | RpcRequestHeaderProto.Builder result = RpcRequestHeaderProto.newBuilder();
160 | result.setRpcKind(convert(rpcKind)).setRpcOp(operation).setCallId(callId)
161 | .setRetryCount(retryCount).setClientId(ByteString.copyFrom(uuid));
162 | + if (clientAddress != null) {
163 | + result.setClientAddress(clientAddress);
164 | + }
165 |
166 | // Add tracing info if we are currently tracing.
167 | if (Trace.isTracing()) {
168 | diff --git a/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto b/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto
169 | index c879150..a95f06b 100644
170 | --- a/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto
171 | +++ b/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto
172 | @@ -80,6 +80,7 @@ message RpcRequestHeaderProto { // the header for the RpcRequest
173 | // retry count, 1 means this is the first retry
174 | optional sint32 retryCount = 5 [default = -1];
175 | optional RPCTraceInfoProto traceInfo = 6; // tracing info
176 | + optional string clientAddress = 7; // real client address, for proxy usage
177 | }
178 |
179 |
180 | diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestProtoUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestProtoUtil.java
181 | index ab891b8..fd4a379 100644
182 | --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestProtoUtil.java
183 | +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestProtoUtil.java
184 | @@ -82,7 +82,7 @@ public void testRpcClientId() {
185 | byte[] uuid = ClientId.getClientId();
186 | RpcRequestHeaderProto header = ProtoUtil.makeRpcRequestHeader(
187 | RpcKind.RPC_PROTOCOL_BUFFER, OperationProto.RPC_FINAL_PACKET, 0,
188 | - RpcConstants.INVALID_RETRY_COUNT, uuid);
189 | + RpcConstants.INVALID_RETRY_COUNT, uuid, null);
190 | assertTrue(Arrays.equals(uuid, header.getClientId().toByteArray()));
191 | }
192 | }
193 | --
194 | 1.9.3 (Apple Git-50)
195 |
196 |
--------------------------------------------------------------------------------
/hadoop-patches/0002-Properly-handle-RemoteException.patch:
--------------------------------------------------------------------------------
1 | From 987b65231a3d1081073c08f80df2eedb3d037546 Mon Sep 17 00:00:00 2001
2 | From: hetianyi
3 | Date: Mon, 14 Mar 2016 22:35:50 +0800
4 | Subject: [PATCH] Properly handle RemoteException
5 |
6 | ---
7 | .../apache/hadoop/ipc/ProxyRpcServerException.java | 32 +++++++++++++++++
8 | .../main/java/org/apache/hadoop/ipc/Server.java | 28 ++++++++++-----
9 | create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProxyRpcServerException.java
10 |
11 | diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProxyRpcServerException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProxyRpcServerException.java
12 | new file mode 100644
13 | index 0000000..812afa8
14 | --- /dev/null
15 | +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProxyRpcServerException.java
16 | @@ -0,0 +1,32 @@
17 | +package org.apache.hadoop.ipc;
18 | +
19 | +import org.apache.hadoop.classification.InterfaceStability;
20 | +import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos;
21 | +
22 | +@InterfaceStability.Evolving
23 | +public class ProxyRpcServerException extends RuntimeException {
24 | +
25 | + final RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto status;
26 | + final RpcHeaderProtos.RpcResponseHeaderProto.RpcErrorCodeProto code;
27 | + final String errorClass;
28 | + final String errorMessage;
29 | +
30 | + public ProxyRpcServerException(RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto status,
31 | + RpcHeaderProtos.RpcResponseHeaderProto.RpcErrorCodeProto code,
32 | + String errorClass, String errorMessage) {
33 | + this.status = status;
34 | + this.code = code;
35 | + this.errorClass = errorClass;
36 | + this.errorMessage = errorMessage;
37 | + }
38 | +
39 | + @Override
40 | + public String toString() {
41 | + return "ProxyRpcServerException [" +
42 | + "status=" + status +
43 | + ", code=" + code +
44 | + ", errorClass=" + errorClass +
45 | + ", errorMessage=" + errorMessage +
46 | + ']';
47 | + }
48 | +}
49 | diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
50 | index 43e8ed4..ce64ddf 100644
51 | --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
52 | +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
53 | @@ -2087,20 +2087,32 @@ public Writable run() throws Exception {
54 | } else {
55 | LOG.info(logMsg, e);
56 | }
57 | - if (e instanceof RpcServerException) {
58 | - RpcServerException rse = ((RpcServerException)e);
59 | + errorClass = null;
60 | + error = null;
61 | + if (e instanceof ProxyRpcServerException) {
62 | + ProxyRpcServerException proxyException = (ProxyRpcServerException)e;
63 | + returnStatus = proxyException.status;
64 | + detailedErr = proxyException.code;
65 | + errorClass = proxyException.errorClass;
66 | + error = proxyException.errorMessage;
67 | + } else if (e instanceof RpcServerException) {
68 | + RpcServerException rse = ((RpcServerException)e);
69 | returnStatus = rse.getRpcStatusProto();
70 | detailedErr = rse.getRpcErrorCodeProto();
71 | } else {
72 | returnStatus = RpcStatusProto.ERROR;
73 | detailedErr = RpcErrorCodeProto.ERROR_APPLICATION;
74 | }
75 | - errorClass = e.getClass().getName();
76 | - error = StringUtils.stringifyException(e);
77 | - // Remove redundant error class name from the beginning of the stack trace
78 | - String exceptionHdr = errorClass + ": ";
79 | - if (error.startsWith(exceptionHdr)) {
80 | - error = error.substring(exceptionHdr.length());
81 | + if (errorClass == null) {
82 | + errorClass = e.getClass().getName();
83 | + }
84 | + if (error == null) {
85 | + error = StringUtils.stringifyException(e);
86 | + // Remove redundant error class name from the beginning of the stack trace
87 | + String exceptionHdr = errorClass + ": ";
88 | + if (error.startsWith(exceptionHdr)) {
89 | + error = error.substring(exceptionHdr.length());
90 | + }
91 | }
92 | }
93 | CurCall.set(null);
94 | --
95 | 1.9.3 (Apple Git-50)
96 |
97 |
--------------------------------------------------------------------------------