Merge trunk into branch.
Conflicts resolved: C hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java ! C hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/SocketCache.java C hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java C hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java C hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java C hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java (thanks to Colin for help resolving) git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-347@1462652 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
694a672131
commit
bbb24fbf5d
|
@ -22,6 +22,10 @@ Trunk (Unreleased)
|
|||
|
||||
HDFS-4296. Reserve layout version for release 1.2.0. (suresh)
|
||||
|
||||
HADOOP-8562. Enhancements to support Hadoop on Windows Server and Windows
|
||||
Azure environments. (See breakdown of tasks below for subtasks and
|
||||
contributors)
|
||||
|
||||
IMPROVEMENTS
|
||||
|
||||
HDFS-1620. Rename HdfsConstants -> HdfsServerConstants, FSConstants ->
|
||||
|
@ -177,6 +181,9 @@ Trunk (Unreleased)
|
|||
HDFS-4346. Add SequentialNumber as a base class for INodeId and
|
||||
GenerationStamp. (szetszwo)
|
||||
|
||||
HDFS-4633 TestDFSClientExcludedNodes fails sporadically if excluded nodes
|
||||
cache expires too quickly (Chris Nauroth via Sanjay)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
@ -299,7 +306,44 @@ Trunk (Unreleased)
|
|||
HDFS-4340. Update addBlock() to inculde inode id as additional argument.
|
||||
(Brandon Li via suresh)
|
||||
|
||||
Release 2.0.4-beta - UNRELEASED
|
||||
HDFS-4502. JsonUtil.toFileStatus(..) should check if the fileId property
|
||||
exists. (Brandon Li via suresh)
|
||||
|
||||
HDFS-4391. TestDataTransferKeepalive fails when tests are executed in a
|
||||
certain order. (Andrew Wang via atm)
|
||||
|
||||
HDFS-4586. TestDataDirs.testGetDataDirsFromURIs fails with all directories
|
||||
in dfs.datanode.data.dir are invalid. (Ivan Mitic via atm)
|
||||
|
||||
BREAKDOWN OF HADOOP-8562 SUBTASKS AND RELATED JIRAS
|
||||
|
||||
HDFS-4145. Merge hdfs cmd line scripts from branch-1-win. (David Lao,
|
||||
Bikas Saha, Lauren Yang, Chuan Liu, Thejas M Nair and Ivan Mitic via suresh)
|
||||
|
||||
HDFS-4163. HDFS distribution build fails on Windows. (Chris Nauroth via
|
||||
suresh)
|
||||
|
||||
HDFS-4316. branch-trunk-win contains test code accidentally added during
|
||||
work on fixing tests on Windows. (Chris Nauroth via suresh)
|
||||
|
||||
HDFS-4297. Fix issues related to datanode concurrent reading and writing on
|
||||
Windows. (Arpit Agarwal, Chuan Liu via suresh)
|
||||
|
||||
HDFS-4573. Fix TestINodeFile on Windows. (Arpit Agarwal via suresh)
|
||||
|
||||
HDFS-4572. Fix TestJournal failures on Windows. (Arpit Agarwal via suresh)
|
||||
|
||||
HDFS-4287. HTTPFS tests fail on Windows. (Chris Nauroth via suresh)
|
||||
|
||||
HDFS-4593. TestSaveNamespace fails on Windows. (Arpit Agarwal via suresh)
|
||||
|
||||
HDFS-4582. TestHostsFiles fails on Windows. (Ivan Mitic via suresh)
|
||||
|
||||
HDFS-4603. TestMiniDFSCluster fails on Windows. (Ivan Mitic via suresh)
|
||||
|
||||
HDFS-4604. TestJournalNode fails on Windows. (Ivan Mitic via suresh)
|
||||
|
||||
Release 2.0.5-beta - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
||||
|
@ -307,6 +351,33 @@ Release 2.0.4-beta - UNRELEASED
|
|||
|
||||
IMPROVEMENTS
|
||||
|
||||
HDFS-4222. NN is unresponsive and loses heartbeats from DNs when
|
||||
configured to use LDAP and LDAP has issues. (Xiaobo Peng, suresh)
|
||||
|
||||
HDFS-4304. Make FSEditLogOp.MAX_OP_SIZE configurable. (Colin Patrick
|
||||
McCabe via atm)
|
||||
|
||||
HDFS-4518. Finer grained metrics for HDFS capacity.
|
||||
(Arpit Agarwal via suresh)
|
||||
|
||||
HDFS-4519. Support overriding jsvc binary and log file locations
|
||||
when launching secure datanode. (Chris Nauroth via suresh)
|
||||
|
||||
HDFS-4569. Small image transfer related cleanups.
|
||||
(Andrew Wang via suresh)
|
||||
|
||||
HDFS-4521. Invalid network toploogies should not be cached. (Colin Patrick
|
||||
McCabe via atm)
|
||||
|
||||
HDFS-4246. The exclude node list should be more forgiving, for each output
|
||||
stream. (harsh via atm)
|
||||
|
||||
HDFS-4635. Move BlockManager#computeCapacity to LightWeightGSet. (suresh)
|
||||
|
||||
HDFS-4621. Additional logging to help diagnose slow QJM syncs. (todd)
|
||||
|
||||
HDFS-4618. Default transaction interval for checkpoints is too low. (todd)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
@ -321,6 +392,86 @@ Release 2.0.4-beta - UNRELEASED
|
|||
but not in dfs.namenode.edits.dir are silently ignored. (Arpit Agarwal
|
||||
via szetszwo)
|
||||
|
||||
HDFS-4482. ReplicationMonitor thread can exit with NPE due to the race
|
||||
between delete and replication of same file. (umamahesh)
|
||||
|
||||
HDFS-4235. When outputting XML, OfflineEditsViewer can't handle some edits
|
||||
containing non-ASCII strings. (Colin Patrick McCabe via atm)
|
||||
|
||||
HDFS-4541. Set hadoop.log.dir and hadoop.id.str when starting secure
|
||||
datanode to write the logs to right dir by default. (Arpit Gupta via
|
||||
suresh)
|
||||
|
||||
HDFS-4540. Namenode http server should use the web authentication
|
||||
keytab for spnego principal. (Arpit Gupta via suresh)
|
||||
|
||||
HDFS-4544. Error in deleting blocks should not do check disk, for
|
||||
all types of errors. (Arpit Agarwal via suresh)
|
||||
|
||||
HDFS-4565. Use DFSUtil.getSpnegoKeytabKey() to get the spnego keytab key
|
||||
in secondary namenode and namenode http server. (Arpit Gupta via suresh)
|
||||
|
||||
HDFS-4571. WebHDFS should not set the service hostname on the server side.
|
||||
(tucu)
|
||||
|
||||
HDFS-4013. TestHftpURLTimeouts throws NPE. (Chao Shi via suresh)
|
||||
|
||||
HDFS-4592. Default values for access time precision are out of sync between
|
||||
hdfs-default.xml and the code. (atm)
|
||||
|
||||
HDFS-4522. LightWeightGSet expects incrementing a volatile to be atomic.
|
||||
(Colin Patrick McCabe via atm)
|
||||
|
||||
HDFS-4484. libwebhdfs compilation broken with gcc 4.6.2. (Colin Patrick
|
||||
McCabe via atm)
|
||||
|
||||
HDFS-4595. When short circuit read is fails, DFSClient does not fallback
|
||||
to regular reads. (suresh)
|
||||
|
||||
HDFS-4583. TestNodeCount fails. (Ivan Mitic via suresh)
|
||||
|
||||
HDFS-4591. HA clients can fail to fail over while Standby NN is performing
|
||||
long checkpoint. (atm)
|
||||
|
||||
HDFS-3277. fail over to loading a different FSImage if the first one we
|
||||
try to load is corrupt. (Colin Patrick McCabe and Andrew Wang via atm)
|
||||
|
||||
HDFS-4596. Shutting down namenode during checkpointing can lead to md5sum
|
||||
error. (Andrew Wang via atm)
|
||||
|
||||
HDFS-4614. FSNamesystem#getContentSummary should use getPermissionChecker
|
||||
helper method. (atm)
|
||||
|
||||
HDFS-4620. Documentation for dfs.namenode.rpc-address specifies wrong
|
||||
format. (Sandy Ryza via atm)
|
||||
|
||||
HDFS-4607. In TestGetConf.testGetSpecificKey(), use a platform-specific
|
||||
line separator; otherwise, it fails on Windows. (Ivan Mitic via szetszwo)
|
||||
|
||||
HDFS-4609. TestAuditLogs should release log handles between tests.
|
||||
(Ivan Mitic via szetszwo)
|
||||
|
||||
HDFS-4615. Fix TestDFSShell failures on Windows. (Arpit Agarwal
|
||||
via szetszwo)
|
||||
|
||||
HDFS-4584. Skip TestNNWithQJM.testNewNamenodeTakesOverWriter() on Windows.
|
||||
(Arpit Agarwal via szetszwo)
|
||||
|
||||
HDFS-4598. Fix the default value of ConcatSourcesParam and the WebHDFS doc.
|
||||
(szetszwo)
|
||||
|
||||
Release 2.0.4-alpha - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
||||
NEW FEATURES
|
||||
|
||||
IMPROVEMENTS
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
||||
Release 2.0.3-alpha - 2013-02-06
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -2292,6 +2443,8 @@ Release 0.23.7 - UNRELEASED
|
|||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HDFS-4532. RPC call queue may fill due to current user lookup (daryn)
|
||||
|
||||
BUG FIXES
|
||||
|
||||
HDFS-4288. NN accepts incremental BR as IBR in safemode (daryn via kihwal)
|
||||
|
@ -2299,6 +2452,31 @@ Release 0.23.7 - UNRELEASED
|
|||
HDFS-4495. Allow client-side lease renewal to be retried beyond soft-limit
|
||||
(kihwal)
|
||||
|
||||
HDFS-4128. 2NN gets stuck in inconsistent state if edit log replay fails
|
||||
in the middle (kihwal via daryn)
|
||||
|
||||
HDFS-4542. Webhdfs doesn't support secure proxy users (Daryn Sharp via
|
||||
kihwal)
|
||||
|
||||
HDFS-4560. Webhdfs cannot use tokens obtained by another user (daryn)
|
||||
|
||||
HDFS-4566. Webdhfs token cancelation should use authentication (daryn)
|
||||
|
||||
HDFS-4567. Webhdfs does not need a token for token operations (daryn via
|
||||
kihwal)
|
||||
|
||||
HDFS-4577. Webhdfs operations should declare if authentication is required
|
||||
(daryn via kihwal)
|
||||
|
||||
HDFS-3344. Unreliable corrupt blocks counting in TestProcessCorruptBlocks
|
||||
(kihwal)
|
||||
|
||||
HDFS-3367. WebHDFS doesn't use the logged in user when opening
|
||||
connections (daryn)
|
||||
|
||||
HDFS-4581. checkDiskError should not be called on network errors (Rohit
|
||||
Kochar via kihwal)
|
||||
|
||||
Release 0.23.6 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -68,8 +68,9 @@ if (NOT GENERATED_JAVAH)
|
|||
MESSAGE(FATAL_ERROR "You must set the CMake variable GENERATED_JAVAH")
|
||||
endif (NOT GENERATED_JAVAH)
|
||||
|
||||
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -g -Wall -O2 -D_GNU_SOURCE")
|
||||
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_REENTRANT -D_FILE_OFFSET_BITS=64")
|
||||
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -g -Wall -O2")
|
||||
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_REENTRANT -D_GNU_SOURCE")
|
||||
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_LARGEFILE_SOURCE -D_FILE_OFFSET_BITS=64")
|
||||
|
||||
include_directories(
|
||||
${GENERATED_JAVAH}
|
||||
|
|
|
@ -163,6 +163,11 @@ class BookKeeperEditLogInputStream extends EditLogInputStream {
|
|||
return ("BookKeeperEditLogInputStream {" + this.getName() + "}");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setMaxOpSize(int maxOpSize) {
|
||||
reader.setMaxOpSize(maxOpSize);
|
||||
}
|
||||
|
||||
/**
|
||||
* Input stream implementation which can be used by
|
||||
* FSEditLogOp.Reader
|
||||
|
|
|
@ -48,6 +48,7 @@ add_executable(test_libwebhdfs_ops
|
|||
)
|
||||
target_link_libraries(test_libwebhdfs_ops
|
||||
webhdfs
|
||||
native_mini_dfs
|
||||
)
|
||||
|
||||
add_executable(test_libwebhdfs_read
|
||||
|
@ -69,4 +70,6 @@ add_executable(test_libwebhdfs_threaded
|
|||
)
|
||||
target_link_libraries(test_libwebhdfs_threaded
|
||||
webhdfs
|
||||
native_mini_dfs
|
||||
pthread
|
||||
)
|
||||
|
|
|
@ -257,7 +257,8 @@ int main(int argc, char **argv)
|
|||
|
||||
const char* path[] = {"/foo", "/foo/bar", "foobar", "//foo/bar//foobar",
|
||||
"foo//bar", "foo/bar///", "/", "////"};
|
||||
for (int i = 0; i < 8; i++) {
|
||||
int i;
|
||||
for (i = 0; i < 8; i++) {
|
||||
fprintf(stderr, "hdfsSetWorkingDirectory: %s, %s\n",
|
||||
((result = hdfsSetWorkingDirectory(fs, path[i])) ?
|
||||
"Failed!" : "Success!"),
|
||||
|
@ -281,8 +282,8 @@ int main(int argc, char **argv)
|
|||
fprintf(stderr, "Name: %s, ", fileInfo->mName);
|
||||
fprintf(stderr, "Type: %c, ", (char)(fileInfo->mKind));
|
||||
fprintf(stderr, "Replication: %d, ", fileInfo->mReplication);
|
||||
fprintf(stderr, "BlockSize: %lld, ", fileInfo->mBlockSize);
|
||||
fprintf(stderr, "Size: %lld, ", fileInfo->mSize);
|
||||
fprintf(stderr, "BlockSize: %"PRId64", ", fileInfo->mBlockSize);
|
||||
fprintf(stderr, "Size: %"PRId64", ", fileInfo->mSize);
|
||||
fprintf(stderr, "LastMod: %s", ctime(&fileInfo->mLastMod));
|
||||
fprintf(stderr, "Owner: %s, ", fileInfo->mOwner);
|
||||
fprintf(stderr, "Group: %s, ", fileInfo->mGroup);
|
||||
|
@ -305,8 +306,8 @@ int main(int argc, char **argv)
|
|||
fprintf(stderr, "Name: %s, ", fileList[i].mName);
|
||||
fprintf(stderr, "Type: %c, ", (char)fileList[i].mKind);
|
||||
fprintf(stderr, "Replication: %d, ", fileList[i].mReplication);
|
||||
fprintf(stderr, "BlockSize: %lld, ", fileList[i].mBlockSize);
|
||||
fprintf(stderr, "Size: %lld, ", fileList[i].mSize);
|
||||
fprintf(stderr, "BlockSize: %"PRId64", ", fileList[i].mBlockSize);
|
||||
fprintf(stderr, "Size: %"PRId64", ", fileList[i].mSize);
|
||||
fprintf(stderr, "LastMod: %s", ctime(&fileList[i].mLastMod));
|
||||
fprintf(stderr, "Owner: %s, ", fileList[i].mOwner);
|
||||
fprintf(stderr, "Group: %s, ", fileList[i].mGroup);
|
||||
|
|
|
@ -15,6 +15,16 @@
|
|||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
# Environment Variables
|
||||
#
|
||||
# JSVC_HOME home directory of jsvc binary. Required for starting secure
|
||||
# datanode.
|
||||
#
|
||||
# JSVC_OUTFILE path to jsvc output file. Defaults to
|
||||
# $HADOOP_LOG_DIR/jsvc.out.
|
||||
#
|
||||
# JSVC_ERRFILE path to jsvc error file. Defaults to $HADOOP_LOG_DIR/jsvc.err.
|
||||
|
||||
bin=`which $0`
|
||||
bin=`dirname ${bin}`
|
||||
bin=`cd "$bin" > /dev/null; pwd`
|
||||
|
@ -56,6 +66,14 @@ fi
|
|||
COMMAND=$1
|
||||
shift
|
||||
|
||||
case $COMMAND in
|
||||
# usage flags
|
||||
--help|-help|-h)
|
||||
print_usage
|
||||
exit
|
||||
;;
|
||||
esac
|
||||
|
||||
# Determine if we're starting a secure datanode, and if so, redefine appropriate variables
|
||||
if [ "$COMMAND" == "datanode" ] && [ "$EUID" -eq 0 ] && [ -n "$HADOOP_SECURE_DN_USER" ]; then
|
||||
if [ -n "$JSVC_HOME" ]; then
|
||||
|
@ -65,9 +83,11 @@ if [ "$COMMAND" == "datanode" ] && [ "$EUID" -eq 0 ] && [ -n "$HADOOP_SECURE_DN_
|
|||
|
||||
if [ -n "$HADOOP_SECURE_DN_LOG_DIR" ]; then
|
||||
HADOOP_LOG_DIR=$HADOOP_SECURE_DN_LOG_DIR
|
||||
HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.log.dir=$HADOOP_LOG_DIR"
|
||||
fi
|
||||
|
||||
HADOOP_IDENT_STRING=$HADOOP_SECURE_DN_USER
|
||||
HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.id.str=$HADOOP_IDENT_STRING"
|
||||
starting_secure_dn="true"
|
||||
else
|
||||
echo "It looks like you're trying to start a secure DN, but \$JSVC_HOME"\
|
||||
|
@ -126,9 +146,6 @@ else
|
|||
CLASS="$COMMAND"
|
||||
fi
|
||||
|
||||
if $cygwin; then
|
||||
CLASSPATH=`cygpath -p -w "$CLASSPATH"`
|
||||
fi
|
||||
export CLASSPATH=$CLASSPATH
|
||||
|
||||
HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.security.logger=${HADOOP_SECURITY_LOGGER:-INFO,NullAppender}"
|
||||
|
@ -148,9 +165,18 @@ if [ "$starting_secure_dn" = "true" ]; then
|
|||
"and set JSVC_HOME to the directory containing the jsvc binary."
|
||||
exit
|
||||
fi
|
||||
|
||||
if [[ ! $JSVC_OUTFILE ]]; then
|
||||
JSVC_OUTFILE="$HADOOP_LOG_DIR/jsvc.out"
|
||||
fi
|
||||
|
||||
if [[ ! $JSVC_ERRFILE ]]; then
|
||||
JSVC_ERRFILE="$HADOOP_LOG_DIR/jsvc.err"
|
||||
fi
|
||||
|
||||
exec "$JSVC" \
|
||||
-Dproc_$COMMAND -outfile "$HADOOP_LOG_DIR/jsvc.out" \
|
||||
-errfile "$HADOOP_LOG_DIR/jsvc.err" \
|
||||
-Dproc_$COMMAND -outfile "$JSVC_OUTFILE" \
|
||||
-errfile "$JSVC_ERRFILE" \
|
||||
-pidfile "$HADOOP_SECURE_DN_PID" \
|
||||
-nodetach \
|
||||
-user "$HADOOP_SECURE_DN_USER" \
|
||||
|
|
|
@ -0,0 +1,43 @@
|
|||
@echo off
|
||||
@rem Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
@rem contributor license agreements. See the NOTICE file distributed with
|
||||
@rem this work for additional information regarding copyright ownership.
|
||||
@rem The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
@rem (the "License"); you may not use this file except in compliance with
|
||||
@rem the License. You may obtain a copy of the License at
|
||||
@rem
|
||||
@rem http://www.apache.org/licenses/LICENSE-2.0
|
||||
@rem
|
||||
@rem Unless required by applicable law or agreed to in writing, software
|
||||
@rem distributed under the License is distributed on an "AS IS" BASIS,
|
||||
@rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
@rem See the License for the specific language governing permissions and
|
||||
@rem limitations under the License.
|
||||
|
||||
@rem included in all the hdfs scripts with source command
|
||||
@rem should not be executed directly
|
||||
|
||||
if not defined HADOOP_BIN_PATH (
|
||||
set HADOOP_BIN_PATH=%~dp0
|
||||
)
|
||||
|
||||
if "%HADOOP_BIN_PATH:~-1%" == "\" (
|
||||
set HADOOP_BIN_PATH=%HADOOP_BIN_PATH:~0,-1%
|
||||
)
|
||||
|
||||
set DEFAULT_LIBEXEC_DIR=%HADOOP_BIN_PATH%\..\libexec
|
||||
if not defined HADOOP_LIBEXEC_DIR (
|
||||
set HADOOP_LIBEXEC_DIR=%DEFAULT_LIBEXEC_DIR%
|
||||
)
|
||||
|
||||
if exist %HADOOP_LIBEXEC_DIR%\hadoop-config.cmd (
|
||||
call %HADOOP_LIBEXEC_DIR%\hadoop-config.cmd %*
|
||||
) else if exist %HADOOP_COMMON_HOME%\libexec\hadoop-config.cmd (
|
||||
call %HADOOP_COMMON_HOME%\libexec\hadoop-config.cmd %*
|
||||
) else if exist %HADOOP_HOME%\libexec\hadoop-config.cmd (
|
||||
call %HADOOP_HOME%\libexec\hadoop-config.cmd %*
|
||||
) else (
|
||||
echo Hadoop common not found.
|
||||
)
|
||||
|
||||
:eof
|
|
@ -0,0 +1,171 @@
|
|||
@echo off
|
||||
@rem Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
@rem contributor license agreements. See the NOTICE file distributed with
|
||||
@rem this work for additional information regarding copyright ownership.
|
||||
@rem The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
@rem (the "License"); you may not use this file except in compliance with
|
||||
@rem the License. You may obtain a copy of the License at
|
||||
@rem
|
||||
@rem http://www.apache.org/licenses/LICENSE-2.0
|
||||
@rem
|
||||
@rem Unless required by applicable law or agreed to in writing, software
|
||||
@rem distributed under the License is distributed on an "AS IS" BASIS,
|
||||
@rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
@rem See the License for the specific language governing permissions and
|
||||
@rem limitations under the License.
|
||||
@rem
|
||||
setlocal enabledelayedexpansion
|
||||
|
||||
if not defined HADOOP_BIN_PATH (
|
||||
set HADOOP_BIN_PATH=%~dp0
|
||||
)
|
||||
|
||||
if "%HADOOP_BIN_PATH:~-1%" == "\" (
|
||||
set HADOOP_BIN_PATH=%HADOOP_BIN_PATH:~0,-1%
|
||||
)
|
||||
|
||||
set DEFAULT_LIBEXEC_DIR=%HADOOP_BIN_PATH%\..\libexec
|
||||
if not defined HADOOP_LIBEXEC_DIR (
|
||||
set HADOOP_LIBEXEC_DIR=%DEFAULT_LIBEXEC_DIR%
|
||||
)
|
||||
|
||||
call %HADOOP_LIBEXEC_DIR%\hdfs-config.cmd %*
|
||||
if "%1" == "--config" (
|
||||
shift
|
||||
shift
|
||||
)
|
||||
|
||||
:main
|
||||
if exist %HADOOP_CONF_DIR%\hadoop-env.cmd (
|
||||
call %HADOOP_CONF_DIR%\hadoop-env.cmd
|
||||
)
|
||||
|
||||
set hdfs-command=%1
|
||||
call :make_command_arguments %*
|
||||
|
||||
if not defined hdfs-command (
|
||||
goto print_usage
|
||||
)
|
||||
|
||||
call :%hdfs-command% %hdfs-command-arguments%
|
||||
set java_arguments=%JAVA_HEAP_MAX% %HADOOP_OPTS% -classpath %CLASSPATH% %CLASS% %hdfs-command-arguments%
|
||||
call %JAVA% %java_arguments%
|
||||
|
||||
goto :eof
|
||||
|
||||
:namenode
|
||||
set CLASS=org.apache.hadoop.hdfs.server.namenode.NameNode
|
||||
set HADOOP_OPTS=%HADOOP_OPTS% %HADOOP_NAMENODE_OPTS%
|
||||
goto :eof
|
||||
|
||||
:zkfc
|
||||
set CLASS=org.apache.hadoop.hdfs.tools.DFSZKFailoverController
|
||||
set HADOOP_OPTS=%HADOOP_OPTS% %HADOOP_ZKFC_OPTS%
|
||||
goto :eof
|
||||
|
||||
:secondarynamenode
|
||||
set CLASS=org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode
|
||||
set HADOOP_OPTS=%HADOOP_OPTS% %HADOOP_SECONDARYNAMENODE_OPTS%
|
||||
goto :eof
|
||||
|
||||
:datanode
|
||||
set CLASS=org.apache.hadoop.hdfs.server.datanode.DataNode
|
||||
set HADOOP_OPTS=%HADOOP_OPTS% -server %HADOOP_DATANODE_OPTS%
|
||||
goto :eof
|
||||
|
||||
:dfs
|
||||
set CLASS=org.apache.hadoop.fs.FsShell
|
||||
set HADOOP_OPTS=%HADOOP_OPTS% %HADOOP_CLIENT_OPTS%
|
||||
goto :eof
|
||||
|
||||
:dfsadmin
|
||||
set CLASS=org.apache.hadoop.hdfs.tools.DFSAdmin
|
||||
set HADOOP_OPTS=%HADOOP_OPTS% %HADOOP_CLIENT_OPTS%
|
||||
goto :eof
|
||||
|
||||
:haadmin
|
||||
set CLASS=org.apache.hadoop.hdfs.tools.DFSHAAdmin
|
||||
set CLASSPATH=%CLASSPATH%;%TOOL_PATH%
|
||||
set HADOOP_OPTS=%HADOOP_OPTS% %HADOOP_CLIENT_OPTS%
|
||||
goto :eof
|
||||
|
||||
:fsck
|
||||
set CLASS=org.apache.hadoop.hdfs.tools.DFSck
|
||||
set HADOOP_OPTS=%HADOOP_OPTS% %HADOOP_CLIENT_OPTS%
|
||||
goto :eof
|
||||
|
||||
:balancer
|
||||
set CLASS=org.apache.hadoop.hdfs.server.balancer.Balancer
|
||||
set HADOOP_OPTS=%HADOOP_OPTS% %HADOOP_BALANCER_OPTS%
|
||||
goto :eof
|
||||
|
||||
:jmxget
|
||||
set CLASS=org.apache.hadoop.hdfs.tools.JMXGet
|
||||
goto :eof
|
||||
|
||||
:oiv
|
||||
set CLASS=org.apache.hadoop.hdfs.tools.offlineImageViewer.OfflineImageViewer
|
||||
goto :eof
|
||||
|
||||
:oev
|
||||
set CLASS=org.apache.hadoop.hdfs.tools.offlineEditsViewer.OfflineEditsViewer
|
||||
goto :eof
|
||||
|
||||
:fetchdt
|
||||
set CLASS=org.apache.hadoop.hdfs.tools.DelegationTokenFetcher
|
||||
goto :eof
|
||||
|
||||
:getconf
|
||||
set CLASS=org.apache.hadoop.hdfs.tools.GetConf
|
||||
goto :eof
|
||||
|
||||
:groups
|
||||
set CLASS=org.apache.hadoop.hdfs.tools.GetGroups
|
||||
goto :eof
|
||||
|
||||
@rem This changes %1, %2 etc. Hence those cannot be used after calling this.
|
||||
:make_command_arguments
|
||||
if "%1" == "--config" (
|
||||
shift
|
||||
shift
|
||||
)
|
||||
if [%2] == [] goto :eof
|
||||
shift
|
||||
set _hdfsarguments=
|
||||
:MakeCmdArgsLoop
|
||||
if [%1]==[] goto :EndLoop
|
||||
|
||||
if not defined _hdfsarguments (
|
||||
set _hdfsarguments=%1
|
||||
) else (
|
||||
set _hdfsarguments=!_hdfsarguments! %1
|
||||
)
|
||||
shift
|
||||
goto :MakeCmdArgsLoop
|
||||
:EndLoop
|
||||
set hdfs-command-arguments=%_hdfsarguments%
|
||||
goto :eof
|
||||
|
||||
:print_usage
|
||||
@echo Usage: hdfs [--config confdir] COMMAND
|
||||
@echo where COMMAND is one of:
|
||||
@echo dfs run a filesystem command on the file systems supported in Hadoop.
|
||||
@echo namenode -format format the DFS filesystem
|
||||
@echo secondarynamenode run the DFS secondary namenode
|
||||
@echo namenode run the DFS namenode
|
||||
@echo zkfc run the ZK Failover Controller daemon
|
||||
@echo datanode run a DFS datanode
|
||||
@echo dfsadmin run a DFS admin client
|
||||
@echo fsck run a DFS filesystem checking utility
|
||||
@echo balancer run a cluster balancing utility
|
||||
@echo jmxget get JMX exported values from NameNode or DataNode.
|
||||
@echo oiv apply the offline fsimage viewer to an fsimage
|
||||
@echo oev apply the offline edits viewer to an edits file
|
||||
@echo fetchdt fetch a delegation token from the NameNode
|
||||
@echo getconf get config values from configuration
|
||||
@echo groups get the groups which users belong to
|
||||
@echo Use -help to see options
|
||||
@echo.
|
||||
@echo Most commands print help when invoked w/o parameters.
|
||||
|
||||
endlocal
|
|
@ -0,0 +1,41 @@
|
|||
@echo off
|
||||
@rem Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
@rem contributor license agreements. See the NOTICE file distributed with
|
||||
@rem this work for additional information regarding copyright ownership.
|
||||
@rem The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
@rem (the "License"); you may not use this file except in compliance with
|
||||
@rem the License. You may obtain a copy of the License at
|
||||
@rem
|
||||
@rem http://www.apache.org/licenses/LICENSE-2.0
|
||||
@rem
|
||||
@rem Unless required by applicable law or agreed to in writing, software
|
||||
@rem distributed under the License is distributed on an "AS IS" BASIS,
|
||||
@rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
@rem See the License for the specific language governing permissions and
|
||||
@rem limitations under the License.
|
||||
@rem
|
||||
setlocal enabledelayedexpansion
|
||||
|
||||
if not defined HADOOP_BIN_PATH (
|
||||
set HADOOP_BIN_PATH=%~dp0
|
||||
)
|
||||
|
||||
if "%HADOOP_BIN_PATH:~-1%" == "\" (
|
||||
set HADOOP_BIN_PATH=%HADOOP_BIN_PATH:~0,-1%
|
||||
)
|
||||
|
||||
set DEFAULT_LIBEXEC_DIR=%HADOOP_BIN_PATH%\..\libexec
|
||||
if not defined HADOOP_LIBEXEC_DIR (
|
||||
set HADOOP_LIBEXEC_DIR=%DEFAULT_LIBEXEC_DIR%
|
||||
)
|
||||
|
||||
call %HADOOP_LIBEXEC_DIR%\hdfs-config.cmd %*
|
||||
if "%1" == "--config" (
|
||||
shift
|
||||
shift
|
||||
)
|
||||
|
||||
start "Apache Hadoop Distribution" hadoop namenode
|
||||
start "Apache Hadoop Distribution" hadoop datanode
|
||||
|
||||
endlocal
|
|
@ -0,0 +1,41 @@
|
|||
@echo off
|
||||
@rem Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
@rem contributor license agreements. See the NOTICE file distributed with
|
||||
@rem this work for additional information regarding copyright ownership.
|
||||
@rem The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
@rem (the "License"); you may not use this file except in compliance with
|
||||
@rem the License. You may obtain a copy of the License at
|
||||
@rem
|
||||
@rem http://www.apache.org/licenses/LICENSE-2.0
|
||||
@rem
|
||||
@rem Unless required by applicable law or agreed to in writing, software
|
||||
@rem distributed under the License is distributed on an "AS IS" BASIS,
|
||||
@rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
@rem See the License for the specific language governing permissions and
|
||||
@rem limitations under the License.
|
||||
@rem
|
||||
setlocal enabledelayedexpansion
|
||||
|
||||
if not defined HADOOP_BIN_PATH (
|
||||
set HADOOP_BIN_PATH=%~dp0
|
||||
)
|
||||
|
||||
if "%HADOOP_BIN_PATH:~-1%" == "\" (
|
||||
set HADOOP_BIN_PATH=%HADOOP_BIN_PATH:~0,-1%
|
||||
)
|
||||
|
||||
set DEFAULT_LIBEXEC_DIR=%HADOOP_BIN_PATH%\..\libexec
|
||||
if not defined HADOOP_LIBEXEC_DIR (
|
||||
set HADOOP_LIBEXEC_DIR=%DEFAULT_LIBEXEC_DIR%
|
||||
)
|
||||
|
||||
call %HADOOP_LIBEXEC_DIR%\hadoop-config.cmd %*
|
||||
if "%1" == "--config" (
|
||||
shift
|
||||
shift
|
||||
)
|
||||
|
||||
Taskkill /FI "WINDOWTITLE eq Apache Hadoop Distribution - hadoop namenode"
|
||||
Taskkill /FI "WINDOWTITLE eq Apache Hadoop Distribution - hadoop datanode"
|
||||
|
||||
endlocal
|
|
@ -76,7 +76,6 @@ See http://forrest.apache.org/docs/linking.html for more info.
|
|||
<zlib href="http://www.zlib.net/" />
|
||||
<gzip href="http://www.gzip.org/" />
|
||||
<bzip href="http://www.bzip.org/" />
|
||||
<cygwin href="http://www.cygwin.com/" />
|
||||
<osx href="http://www.apple.com/macosx" />
|
||||
<hod href="">
|
||||
<cluster-resources href="http://www.clusterresources.com" />
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.hadoop.io.IOUtils;
|
|||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
|
||||
|
@ -242,14 +243,15 @@ public class BlockReaderFactory {
|
|||
* This block reader implements the path-based style of local reads
|
||||
* first introduced in HDFS-2246.
|
||||
*/
|
||||
static BlockReader getLegacyBlockReaderLocal(Configuration conf,
|
||||
String src, ExtendedBlock blk, Token<BlockTokenIdentifier> accessToken,
|
||||
DatanodeInfo chosenNode, int socketTimeout, long offsetIntoBlock,
|
||||
static BlockReader getLegacyBlockReaderLocal(UserGroupInformation ugi,
|
||||
Configuration conf, String src, ExtendedBlock blk,
|
||||
Token<BlockTokenIdentifier> accessToken, DatanodeInfo chosenNode,
|
||||
int socketTimeout, long offsetIntoBlock,
|
||||
boolean connectToDnViaHostname) throws InvalidToken, IOException {
|
||||
try {
|
||||
return BlockReaderLocalLegacy.newBlockReader(conf, src, blk, accessToken,
|
||||
chosenNode, socketTimeout, offsetIntoBlock, blk.getNumBytes()
|
||||
- offsetIntoBlock, connectToDnViaHostname);
|
||||
return BlockReaderLocalLegacy.newBlockReader(ugi, conf, src,
|
||||
blk, accessToken, chosenNode, socketTimeout, offsetIntoBlock,
|
||||
blk.getNumBytes() - offsetIntoBlock, connectToDnViaHostname);
|
||||
} catch (RemoteException re) {
|
||||
throw re.unwrapRemoteException(InvalidToken.class,
|
||||
AccessControlException.class);
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.io.FileInputStream;
|
|||
import java.io.IOException;
|
||||
import java.net.Socket;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
|
@ -31,6 +32,7 @@ import java.util.Map;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
|
@ -41,6 +43,7 @@ import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
|||
import org.apache.hadoop.hdfs.util.DirectBufferPool;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
|
||||
|
@ -92,11 +95,21 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
}
|
||||
|
||||
private synchronized ClientDatanodeProtocol getDatanodeProxy(
|
||||
DatanodeInfo node, Configuration conf, int socketTimeout,
|
||||
boolean connectToDnViaHostname) throws IOException {
|
||||
UserGroupInformation ugi, final DatanodeInfo node,
|
||||
final Configuration conf, final int socketTimeout,
|
||||
final boolean connectToDnViaHostname) throws IOException {
|
||||
if (proxy == null) {
|
||||
proxy = DFSUtil.createClientDatanodeProtocolProxy(node, conf,
|
||||
socketTimeout, connectToDnViaHostname);
|
||||
try {
|
||||
proxy = ugi.doAs(new PrivilegedExceptionAction<ClientDatanodeProtocol>() {
|
||||
@Override
|
||||
public ClientDatanodeProtocol run() throws Exception {
|
||||
return DFSUtil.createClientDatanodeProtocolProxy(node, conf,
|
||||
socketTimeout, connectToDnViaHostname);
|
||||
}
|
||||
});
|
||||
} catch (InterruptedException e) {
|
||||
LOG.warn("encountered exception ", e);
|
||||
}
|
||||
}
|
||||
return proxy;
|
||||
}
|
||||
|
@ -160,17 +173,18 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
/**
|
||||
* The only way this object can be instantiated.
|
||||
*/
|
||||
static BlockReaderLocalLegacy newBlockReader(Configuration conf, String file,
|
||||
ExtendedBlock blk, Token<BlockTokenIdentifier> token, DatanodeInfo node,
|
||||
int socketTimeout, long startOffset, long length,
|
||||
boolean connectToDnViaHostname) throws IOException {
|
||||
static BlockReaderLocalLegacy newBlockReader(UserGroupInformation ugi,
|
||||
Configuration conf, String file, ExtendedBlock blk,
|
||||
Token<BlockTokenIdentifier> token, DatanodeInfo node, int socketTimeout,
|
||||
long startOffset, long length, boolean connectToDnViaHostname)
|
||||
throws IOException {
|
||||
|
||||
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
|
||||
.getIpcPort());
|
||||
// check the cache first
|
||||
BlockLocalPathInfo pathinfo = localDatanodeInfo.getBlockLocalPathInfo(blk);
|
||||
if (pathinfo == null) {
|
||||
pathinfo = getBlockPathInfo(blk, node, conf, socketTimeout, token,
|
||||
pathinfo = getBlockPathInfo(ugi, blk, node, conf, socketTimeout, token,
|
||||
connectToDnViaHostname);
|
||||
}
|
||||
|
||||
|
@ -247,13 +261,13 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
return ldInfo;
|
||||
}
|
||||
|
||||
private static BlockLocalPathInfo getBlockPathInfo(ExtendedBlock blk,
|
||||
DatanodeInfo node, Configuration conf, int timeout,
|
||||
private static BlockLocalPathInfo getBlockPathInfo(UserGroupInformation ugi,
|
||||
ExtendedBlock blk, DatanodeInfo node, Configuration conf, int timeout,
|
||||
Token<BlockTokenIdentifier> token, boolean connectToDnViaHostname)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node.getIpcPort());
|
||||
BlockLocalPathInfo pathinfo = null;
|
||||
ClientDatanodeProtocol proxy = localDatanodeInfo.getDatanodeProxy(node,
|
||||
ClientDatanodeProtocol proxy = localDatanodeInfo.getDatanodeProxy(ugi, node,
|
||||
conf, timeout, connectToDnViaHostname);
|
||||
try {
|
||||
// make RPC to local datanode to find local pathnames of blocks
|
||||
|
@ -282,9 +296,11 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT);
|
||||
|
||||
if (bufferSizeBytes < bytesPerChecksum) {
|
||||
throw new IllegalArgumentException("Configured BlockReaderLocalLegacy buffer size (" + bufferSizeBytes + ") " +
|
||||
"is not large enough to hold a single chunk (" + bytesPerChecksum + "). Please configure " +
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY + " appropriately");
|
||||
throw new IllegalArgumentException("Configured BlockReaderLocalLegacy " +
|
||||
"buffer size (" + bufferSizeBytes + ") is not large enough to hold " +
|
||||
"a single chunk (" + bytesPerChecksum + "). Please configure " +
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY +
|
||||
" appropriately");
|
||||
}
|
||||
|
||||
// Round down to nearest chunk size
|
||||
|
|
|
@ -41,6 +41,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPAC
|
|||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER_DEFAULT;
|
||||
|
@ -212,6 +214,7 @@ public class DFSClient implements java.io.Closeable {
|
|||
final int socketTimeout;
|
||||
final int socketCacheCapacity;
|
||||
final long socketCacheExpiry;
|
||||
final long excludedNodesCacheExpiry;
|
||||
/** Wait time window (in msec) if BlockMissingException is caught */
|
||||
final int timeWindow;
|
||||
final int nCachedConnRetry;
|
||||
|
@ -267,6 +270,9 @@ public class DFSClient implements java.io.Closeable {
|
|||
DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT);
|
||||
socketCacheExpiry = conf.getLong(DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY,
|
||||
DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT);
|
||||
excludedNodesCacheExpiry = conf.getLong(
|
||||
DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL,
|
||||
DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT);
|
||||
prefetchSize = conf.getLong(DFS_CLIENT_READ_PREFETCH_SIZE_KEY,
|
||||
10 * defaultBlockSize);
|
||||
timeWindow = conf
|
||||
|
@ -436,6 +442,7 @@ public class DFSClient implements java.io.Closeable {
|
|||
"null URI");
|
||||
NameNodeProxies.ProxyAndInfo<ClientProtocol> proxyInfo =
|
||||
NameNodeProxies.createProxy(conf, nameNodeUri, ClientProtocol.class);
|
||||
|
||||
this.dtService = proxyInfo.getDelegationTokenService();
|
||||
this.namenode = proxyInfo.getProxy();
|
||||
}
|
||||
|
@ -1614,7 +1621,7 @@ public class DFSClient implements java.io.Closeable {
|
|||
* @param socketFactory to create sockets to connect to DNs
|
||||
* @param socketTimeout timeout to use when connecting and waiting for a response
|
||||
* @param encryptionKey the key needed to communicate with DNs in this cluster
|
||||
* @param connectToDnViaHostname {@see #connectToDnViaHostname()}
|
||||
* @param connectToDnViaHostname {@link #connectToDnViaHostname()}
|
||||
* @return The checksum
|
||||
*/
|
||||
static MD5MD5CRC32FileChecksum getFileChecksum(String src,
|
||||
|
|
|
@ -76,6 +76,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||
|
||||
public static final String DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY = "dfs.client.socketcache.expiryMsec";
|
||||
public static final long DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT = 2 * 60 * 1000;
|
||||
public static final String DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL = "dfs.client.write.exclude.nodes.cache.expiry.interval.millis";
|
||||
public static final long DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT = 10 * 60 * 1000; // 10 minutes, in ms
|
||||
public static final String DFS_NAMENODE_BACKUP_ADDRESS_KEY = "dfs.namenode.backup.address";
|
||||
public static final String DFS_NAMENODE_BACKUP_ADDRESS_DEFAULT = "localhost:50100";
|
||||
public static final String DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY = "dfs.namenode.backup.http-address";
|
||||
|
@ -120,7 +122,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||
public static final String DFS_NAMENODE_CHECKPOINT_PERIOD_KEY = "dfs.namenode.checkpoint.period";
|
||||
public static final long DFS_NAMENODE_CHECKPOINT_PERIOD_DEFAULT = 3600;
|
||||
public static final String DFS_NAMENODE_CHECKPOINT_TXNS_KEY = "dfs.namenode.checkpoint.txns";
|
||||
public static final long DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT = 40000;
|
||||
public static final long DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT = 1000000;
|
||||
public static final String DFS_NAMENODE_CHECKPOINT_MAX_RETRIES_KEY = "dfs.namenode.checkpoint.max-retries";
|
||||
public static final int DFS_NAMENODE_CHECKPOINT_MAX_RETRIES_DEFAULT = 3;
|
||||
public static final String DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY = "dfs.namenode.heartbeat.recheck-interval";
|
||||
public static final int DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT = 5*60*1000;
|
||||
public static final String DFS_NAMENODE_TOLERATE_HEARTBEAT_MULTIPLIER_KEY = "dfs.namenode.tolerate.heartbeat.multiplier";
|
||||
|
@ -367,7 +371,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||
|
||||
// Image transfer timeout
|
||||
public static final String DFS_IMAGE_TRANSFER_TIMEOUT_KEY = "dfs.image.transfer.timeout";
|
||||
public static final int DFS_IMAGE_TRANSFER_TIMEOUT_DEFAULT = 60 * 1000;
|
||||
public static final int DFS_IMAGE_TRANSFER_TIMEOUT_DEFAULT = 10 * 60 * 1000;
|
||||
|
||||
//Keys with no defaults
|
||||
public static final String DFS_DATANODE_PLUGINS_KEY = "dfs.datanode.plugins";
|
||||
|
@ -400,6 +404,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||
public static final int DFS_NAMENODE_CHECKED_VOLUMES_MINIMUM_DEFAULT = 1;
|
||||
public static final String DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY = "dfs.web.authentication.kerberos.principal";
|
||||
public static final String DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY = "dfs.web.authentication.kerberos.keytab";
|
||||
public static final String DFS_NAMENODE_MAX_OP_SIZE_KEY = "dfs.namenode.max.op.size";
|
||||
public static final int DFS_NAMENODE_MAX_OP_SIZE_DEFAULT = 50 * 1024 * 1024;
|
||||
|
||||
public static final String DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY = "dfs.block.local-path-access.user";
|
||||
public static final String DFS_DOMAIN_SOCKET_PATH_KEY = "dfs.domain.socket.path";
|
||||
|
|
|
@ -55,6 +55,7 @@ import org.apache.hadoop.ipc.RPC;
|
|||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
@ -476,6 +477,10 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|||
" for " + blk);
|
||||
}
|
||||
return chosenNode;
|
||||
} catch (AccessControlException ex) {
|
||||
DFSClient.LOG.warn("Short circuit access failed " + ex);
|
||||
dfsClient.disableLegacyBlockReaderLocal();
|
||||
continue;
|
||||
} catch (IOException ex) {
|
||||
if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
|
||||
DFSClient.LOG.info("Will fetch a new encryption key and retry, "
|
||||
|
@ -822,6 +827,10 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|||
e.getPos() + " from " + chosenNode);
|
||||
// we want to remember what we have tried
|
||||
addIntoCorruptedBlockMap(block.getBlock(), chosenNode, corruptedBlockMap);
|
||||
} catch (AccessControlException ex) {
|
||||
DFSClient.LOG.warn("Short circuit access failed " + ex);
|
||||
dfsClient.disableLegacyBlockReaderLocal();
|
||||
continue;
|
||||
} catch (IOException e) {
|
||||
if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
|
||||
DFSClient.LOG.info("Will fetch a new encryption key and retry, "
|
||||
|
@ -925,8 +934,8 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|||
DFSClient.isLocalAddress(dnAddr) &&
|
||||
(!shortCircuitForbidden())) {
|
||||
try {
|
||||
return BlockReaderFactory.getLegacyBlockReaderLocal(dfsClient.conf,
|
||||
clientName, block, blockToken, chosenNode,
|
||||
return BlockReaderFactory.getLegacyBlockReaderLocal(dfsClient.ugi,
|
||||
dfsClient.conf, clientName, block, blockToken, chosenNode,
|
||||
dfsClient.hdfsTimeout, startOffset,dfsClient.connectToDnViaHostname());
|
||||
} catch (IOException e) {
|
||||
DFSClient.LOG.warn("error creating legacy BlockReaderLocal. " +
|
||||
|
@ -1083,8 +1092,8 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|||
* only report if the total number of replica is 1. We do not
|
||||
* report otherwise since this maybe due to the client is a handicapped client
|
||||
* (who can not read).
|
||||
* @param corruptedBlockMap, map of corrupted blocks
|
||||
* @param dataNodeCount, number of data nodes who contains the block replicas
|
||||
* @param corruptedBlockMap map of corrupted blocks
|
||||
* @param dataNodeCount number of data nodes who contains the block replicas
|
||||
*/
|
||||
private void reportCheckSumFailure(
|
||||
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
|
||||
|
|
|
@ -35,6 +35,7 @@ import java.util.Arrays;
|
|||
import java.util.EnumSet;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
@ -82,6 +83,11 @@ import org.apache.hadoop.util.Progressable;
|
|||
import org.apache.hadoop.util.Time;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.cache.CacheBuilder;
|
||||
import com.google.common.cache.CacheLoader;
|
||||
import com.google.common.cache.LoadingCache;
|
||||
import com.google.common.cache.RemovalListener;
|
||||
import com.google.common.cache.RemovalNotification;
|
||||
|
||||
|
||||
/****************************************************************
|
||||
|
@ -289,7 +295,25 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
|
|||
private DataInputStream blockReplyStream;
|
||||
private ResponseProcessor response = null;
|
||||
private volatile DatanodeInfo[] nodes = null; // list of targets for current block
|
||||
private ArrayList<DatanodeInfo> excludedNodes = new ArrayList<DatanodeInfo>();
|
||||
private LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes =
|
||||
CacheBuilder.newBuilder()
|
||||
.expireAfterWrite(
|
||||
dfsClient.getConf().excludedNodesCacheExpiry,
|
||||
TimeUnit.MILLISECONDS)
|
||||
.removalListener(new RemovalListener<DatanodeInfo, DatanodeInfo>() {
|
||||
@Override
|
||||
public void onRemoval(
|
||||
RemovalNotification<DatanodeInfo, DatanodeInfo> notification) {
|
||||
DFSClient.LOG.info("Removing node " +
|
||||
notification.getKey() + " from the excluded nodes list");
|
||||
}
|
||||
})
|
||||
.build(new CacheLoader<DatanodeInfo, DatanodeInfo>() {
|
||||
@Override
|
||||
public DatanodeInfo load(DatanodeInfo key) throws Exception {
|
||||
return key;
|
||||
}
|
||||
});
|
||||
volatile boolean hasError = false;
|
||||
volatile int errorIndex = -1;
|
||||
private BlockConstructionStage stage; // block construction stage
|
||||
|
@ -999,8 +1023,10 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
|
|||
success = false;
|
||||
|
||||
long startTime = Time.now();
|
||||
DatanodeInfo[] excluded = excludedNodes.toArray(
|
||||
new DatanodeInfo[excludedNodes.size()]);
|
||||
DatanodeInfo[] excluded =
|
||||
excludedNodes.getAllPresent(excludedNodes.asMap().keySet())
|
||||
.keySet()
|
||||
.toArray(new DatanodeInfo[0]);
|
||||
block = oldBlock;
|
||||
lb = locateFollowingBlock(startTime,
|
||||
excluded.length > 0 ? excluded : null);
|
||||
|
@ -1019,7 +1045,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
|
|||
dfsClient.namenode.abandonBlock(block, src, dfsClient.clientName);
|
||||
block = null;
|
||||
DFSClient.LOG.info("Excluding datanode " + nodes[errorIndex]);
|
||||
excludedNodes.add(nodes[errorIndex]);
|
||||
excludedNodes.put(nodes[errorIndex], nodes[errorIndex]);
|
||||
}
|
||||
} while (!success && --count >= 0);
|
||||
|
||||
|
|
|
@ -1259,4 +1259,20 @@ public class DFSUtil {
|
|||
"It should be a positive, non-zero integer value.");
|
||||
return blocksReplWorkMultiplier;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get SPNEGO keytab Key from configuration
|
||||
*
|
||||
* @param conf
|
||||
* Configuration
|
||||
* @param defaultKey
|
||||
* @return DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY if the key is not empty
|
||||
* else return defaultKey
|
||||
*/
|
||||
public static String getSpnegoKeytabKey(Configuration conf, String defaultKey) {
|
||||
String value =
|
||||
conf.get(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY);
|
||||
return (value == null || value.isEmpty()) ?
|
||||
defaultKey : DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY;
|
||||
}
|
||||
}
|
|
@ -62,11 +62,14 @@ import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
|
|||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.Progressable;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
|
||||
/****************************************************************
|
||||
* Implementation of the abstract FileSystem for the DFS system.
|
||||
|
@ -310,13 +313,14 @@ public class DistributedFileSystem extends FileSystem {
|
|||
}
|
||||
|
||||
/**
|
||||
* Move blocks from srcs to trg
|
||||
* and delete srcs afterwards
|
||||
* RESTRICTION: all blocks should be the same size
|
||||
* Move blocks from srcs to trg and delete srcs afterwards.
|
||||
* The file block sizes must be the same.
|
||||
*
|
||||
* @param trg existing file to append to
|
||||
* @param psrcs list of files (same block size, same replication)
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public void concat(Path trg, Path [] psrcs) throws IOException {
|
||||
String [] srcs = new String [psrcs.length];
|
||||
for(int i=0; i<psrcs.length; i++) {
|
||||
|
@ -564,9 +568,8 @@ public class DistributedFileSystem extends FileSystem {
|
|||
return "DFS[" + dfs + "]";
|
||||
}
|
||||
|
||||
/** @deprecated DFSClient should not be accessed directly. */
|
||||
@InterfaceAudience.Private
|
||||
@Deprecated
|
||||
@VisibleForTesting
|
||||
public DFSClient getClient() {
|
||||
return dfs;
|
||||
}
|
||||
|
@ -891,6 +894,17 @@ public class DistributedFileSystem extends FileSystem {
|
|||
public String getCanonicalServiceName() {
|
||||
return dfs.getCanonicalServiceName();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected URI canonicalizeUri(URI uri) {
|
||||
if (HAUtil.isLogicalUri(getConf(), uri)) {
|
||||
// Don't try to DNS-resolve logical URIs, since the 'authority'
|
||||
// portion isn't a proper hostname
|
||||
return uri;
|
||||
} else {
|
||||
return NetUtils.getCanonicalUri(uri, getDefaultPort());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility function that returns if the NameNode is in safemode or not. In HA
|
||||
|
|
|
@ -161,6 +161,11 @@ public class HftpFileSystem extends FileSystem
|
|||
// actual port in the uri
|
||||
return SecurityUtil.buildTokenService(nnSecureUri).toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected URI canonicalizeUri(URI uri) {
|
||||
return NetUtils.getCanonicalUri(uri, getDefaultPort());
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the protocol scheme for the FileSystem.
|
||||
|
|
|
@ -133,6 +133,8 @@ public class IPCLoggerChannel implements AsyncLogger {
|
|||
private Stopwatch lastHeartbeatStopwatch = new Stopwatch();
|
||||
|
||||
private static final long HEARTBEAT_INTERVAL_MILLIS = 1000;
|
||||
|
||||
private static final long WARN_JOURNAL_MILLIS_THRESHOLD = 1000;
|
||||
|
||||
static final Factory FACTORY = new AsyncLogger.Factory() {
|
||||
@Override
|
||||
|
@ -371,6 +373,12 @@ public class IPCLoggerChannel implements AsyncLogger {
|
|||
now - submitNanos, TimeUnit.NANOSECONDS);
|
||||
metrics.addWriteEndToEndLatency(endToEndTime);
|
||||
metrics.addWriteRpcLatency(rpcTime);
|
||||
if (rpcTime / 1000 > WARN_JOURNAL_MILLIS_THRESHOLD) {
|
||||
QuorumJournalManager.LOG.warn(
|
||||
"Took " + (rpcTime / 1000) + "ms to send a batch of " +
|
||||
numTxns + " edits (" + data.length + " bytes) to " +
|
||||
"remote journal " + IPCLoggerChannel.this);
|
||||
}
|
||||
}
|
||||
synchronized (IPCLoggerChannel.this) {
|
||||
highestAckedTxId = firstTxnId + numTxns - 1;
|
||||
|
|
|
@ -24,6 +24,7 @@ import java.util.concurrent.TimeoutException;
|
|||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.util.concurrent.FutureCallback;
|
||||
|
@ -120,6 +121,15 @@ class QuorumCall<KEY, RESULT> {
|
|||
String msg = String.format(
|
||||
"Waited %s ms (timeout=%s ms) for a response for %s",
|
||||
waited, millis, operationName);
|
||||
if (!successes.isEmpty()) {
|
||||
msg += ". Succeeded so far: [" + Joiner.on(",").join(successes.keySet()) + "]";
|
||||
}
|
||||
if (!exceptions.isEmpty()) {
|
||||
msg += ". Exceptions so far: [" + getExceptionMapString() + "]";
|
||||
}
|
||||
if (successes.isEmpty() && exceptions.isEmpty()) {
|
||||
msg += ". No responses yet.";
|
||||
}
|
||||
if (waited > millis * WAIT_PROGRESS_WARN_THRESHOLD) {
|
||||
QuorumJournalManager.LOG.warn(msg);
|
||||
} else {
|
||||
|
@ -227,4 +237,22 @@ class QuorumCall<KEY, RESULT> {
|
|||
}
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a string suitable for displaying to the user, containing
|
||||
* any exceptions that have been received so far.
|
||||
*/
|
||||
private String getExceptionMapString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
boolean first = true;
|
||||
for (Map.Entry<KEY, Throwable> e : exceptions.entrySet()) {
|
||||
if (!first) {
|
||||
sb.append(", ");
|
||||
}
|
||||
first = false;
|
||||
sb.append(e.getKey()).append(": ")
|
||||
.append(e.getValue().getLocalizedMessage());
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -128,6 +128,10 @@ class Journal implements Closeable {
|
|||
|
||||
private final JournalMetrics metrics;
|
||||
|
||||
/**
|
||||
* Time threshold for sync calls, beyond which a warning should be logged to the console.
|
||||
*/
|
||||
private static final int WARN_SYNC_MILLIS_THRESHOLD = 1000;
|
||||
|
||||
Journal(File logDir, String journalId,
|
||||
StorageErrorReporter errorReporter) throws IOException {
|
||||
|
@ -215,8 +219,8 @@ class Journal implements Closeable {
|
|||
@Override // Closeable
|
||||
public void close() throws IOException {
|
||||
storage.close();
|
||||
|
||||
IOUtils.closeStream(committedTxnId);
|
||||
IOUtils.closeStream(curSegment);
|
||||
}
|
||||
|
||||
JNStorage getStorage() {
|
||||
|
@ -370,6 +374,10 @@ class Journal implements Closeable {
|
|||
sw.stop();
|
||||
|
||||
metrics.addSync(sw.elapsedTime(TimeUnit.MICROSECONDS));
|
||||
if (sw.elapsedTime(TimeUnit.MILLISECONDS) > WARN_SYNC_MILLIS_THRESHOLD) {
|
||||
LOG.warn("Sync of transaction range " + firstTxnId + "-" + lastTxnId +
|
||||
" took " + sw.elapsedTime(TimeUnit.MILLISECONDS) + "ms");
|
||||
}
|
||||
|
||||
if (isLagging) {
|
||||
// This batch of edits has already been committed on a quorum of other
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.hadoop.io.IOUtils;
|
|||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||
import org.apache.hadoop.metrics2.source.JvmMetrics;
|
||||
import org.apache.hadoop.security.SecurityUtil;
|
||||
import org.apache.hadoop.util.DiskChecker;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
|
@ -82,7 +83,6 @@ public class JournalNode implements Tool, Configurable {
|
|||
return journal;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void setConf(Configuration conf) {
|
||||
this.conf = conf;
|
||||
|
@ -97,21 +97,9 @@ public class JournalNode implements Tool, Configurable {
|
|||
"Journal dir '" + dir + "' should be an absolute path");
|
||||
}
|
||||
|
||||
if (!dir.exists() && !dir.mkdirs()) {
|
||||
throw new IOException("Could not create journal dir '" +
|
||||
dir + "'");
|
||||
} else if (!dir.isDirectory()) {
|
||||
throw new IOException("Journal directory '" + dir + "' is not " +
|
||||
"a directory");
|
||||
}
|
||||
|
||||
if (!dir.canWrite()) {
|
||||
throw new IOException("Unable to write to journal dir '" +
|
||||
dir + "'");
|
||||
}
|
||||
DiskChecker.checkDir(dir);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Configuration getConf() {
|
||||
return conf;
|
||||
|
|
|
@ -78,6 +78,7 @@ public class DelegationTokenSecretManager
|
|||
|
||||
@Override //SecretManager
|
||||
public void checkAvailableForRead() throws StandbyException {
|
||||
namesystem.checkOperation(OperationCategory.READ);
|
||||
namesystem.readLock();
|
||||
try {
|
||||
namesystem.checkOperation(OperationCategory.READ);
|
||||
|
|
|
@ -62,6 +62,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|||
import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||
import org.apache.hadoop.hdfs.server.namenode.Namesystem;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
|
||||
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
|
||||
|
@ -234,6 +235,7 @@ public class BlockManager {
|
|||
heartbeatManager = datanodeManager.getHeartbeatManager();
|
||||
invalidateBlocks = new InvalidateBlocks(datanodeManager);
|
||||
|
||||
// Compute the map capacity by allocating 2% of total memory
|
||||
blocksMap = new BlocksMap(DEFAULT_MAP_LOAD_FACTOR);
|
||||
blockplacement = BlockPlacementPolicy.getInstance(
|
||||
conf, stats, datanodeManager.getNetworkTopology());
|
||||
|
@ -874,9 +876,10 @@ public class BlockManager {
|
|||
*/
|
||||
public BlocksWithLocations getBlocks(DatanodeID datanode, long size
|
||||
) throws IOException {
|
||||
namesystem.checkOperation(OperationCategory.READ);
|
||||
namesystem.readLock();
|
||||
try {
|
||||
namesystem.checkSuperuserPrivilege();
|
||||
namesystem.checkOperation(OperationCategory.READ);
|
||||
return getBlocksWithLocations(datanode, size);
|
||||
} finally {
|
||||
namesystem.readUnlock();
|
||||
|
|
|
@ -60,38 +60,11 @@ class BlocksMap {
|
|||
private GSet<Block, BlockInfo> blocks;
|
||||
|
||||
BlocksMap(final float loadFactor) {
|
||||
this.capacity = computeCapacity();
|
||||
// Use 2% of total memory to size the GSet capacity
|
||||
this.capacity = LightWeightGSet.computeCapacity(2.0, "BlocksMap");
|
||||
this.blocks = new LightWeightGSet<Block, BlockInfo>(capacity);
|
||||
}
|
||||
|
||||
/**
|
||||
* Let t = 2% of max memory.
|
||||
* Let e = round(log_2 t).
|
||||
* Then, we choose capacity = 2^e/(size of reference),
|
||||
* unless it is outside the close interval [1, 2^30].
|
||||
*/
|
||||
private static int computeCapacity() {
|
||||
//VM detection
|
||||
//See http://java.sun.com/docs/hotspot/HotSpotFAQ.html#64bit_detection
|
||||
final String vmBit = System.getProperty("sun.arch.data.model");
|
||||
|
||||
//2% of max memory
|
||||
final double twoPC = Runtime.getRuntime().maxMemory()/50.0;
|
||||
|
||||
//compute capacity
|
||||
final int e1 = (int)(Math.log(twoPC)/Math.log(2.0) + 0.5);
|
||||
final int e2 = e1 - ("32".equals(vmBit)? 2: 3);
|
||||
final int exponent = e2 < 0? 0: e2 > 30? 30: e2;
|
||||
final int c = 1 << exponent;
|
||||
|
||||
if (LightWeightGSet.LOG.isDebugEnabled()) {
|
||||
LightWeightGSet.LOG.debug("VM type = " + vmBit + "-bit");
|
||||
LightWeightGSet.LOG.debug("2% max memory = " + twoPC/(1 << 20) + " MB");
|
||||
LightWeightGSet.LOG.debug("capacity = 2^" + exponent
|
||||
+ " = " + c + " entries");
|
||||
}
|
||||
return c;
|
||||
}
|
||||
|
||||
void close() {
|
||||
// Empty blocks once GSet#clear is implemented (HDFS-3940)
|
||||
|
|
|
@ -66,6 +66,7 @@ import org.apache.hadoop.ipc.Server;
|
|||
import org.apache.hadoop.net.CachedDNSToSwitchMapping;
|
||||
import org.apache.hadoop.net.DNSToSwitchMapping;
|
||||
import org.apache.hadoop.net.NetworkTopology;
|
||||
import org.apache.hadoop.net.NetworkTopology.InvalidTopologyException;
|
||||
import org.apache.hadoop.net.Node;
|
||||
import org.apache.hadoop.net.NodeBase;
|
||||
import org.apache.hadoop.net.ScriptBasedMapping;
|
||||
|
@ -431,8 +432,8 @@ public class DatanodeManager {
|
|||
host2DatanodeMap.remove(datanodeMap.put(node.getStorageID(), node));
|
||||
}
|
||||
|
||||
networktopology.add(node); // may throw InvalidTopologyException
|
||||
host2DatanodeMap.add(node);
|
||||
networktopology.add(node);
|
||||
checkIfClusterIsNowMultiRack(node);
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
|
@ -647,92 +648,122 @@ public class DatanodeManager {
|
|||
nodeReg.setIpAddr(ip);
|
||||
nodeReg.setPeerHostName(hostname);
|
||||
}
|
||||
|
||||
nodeReg.setExportedKeys(blockManager.getBlockKeys());
|
||||
|
||||
// Checks if the node is not on the hosts list. If it is not, then
|
||||
// it will be disallowed from registering.
|
||||
if (!inHostsList(nodeReg)) {
|
||||
throw new DisallowedDatanodeException(nodeReg);
|
||||
}
|
||||
|
||||
NameNode.stateChangeLog.info("BLOCK* registerDatanode: from "
|
||||
+ nodeReg + " storage " + nodeReg.getStorageID());
|
||||
|
||||
DatanodeDescriptor nodeS = datanodeMap.get(nodeReg.getStorageID());
|
||||
DatanodeDescriptor nodeN = host2DatanodeMap.getDatanodeByXferAddr(
|
||||
nodeReg.getIpAddr(), nodeReg.getXferPort());
|
||||
|
||||
if (nodeN != null && nodeN != nodeS) {
|
||||
NameNode.LOG.info("BLOCK* registerDatanode: " + nodeN);
|
||||
// nodeN previously served a different data storage,
|
||||
// which is not served by anybody anymore.
|
||||
removeDatanode(nodeN);
|
||||
// physically remove node from datanodeMap
|
||||
wipeDatanode(nodeN);
|
||||
nodeN = null;
|
||||
}
|
||||
|
||||
if (nodeS != null) {
|
||||
if (nodeN == nodeS) {
|
||||
// The same datanode has been just restarted to serve the same data
|
||||
// storage. We do not need to remove old data blocks, the delta will
|
||||
// be calculated on the next block report from the datanode
|
||||
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug("BLOCK* registerDatanode: "
|
||||
+ "node restarted.");
|
||||
}
|
||||
} else {
|
||||
// nodeS is found
|
||||
/* The registering datanode is a replacement node for the existing
|
||||
data storage, which from now on will be served by a new node.
|
||||
If this message repeats, both nodes might have same storageID
|
||||
by (insanely rare) random chance. User needs to restart one of the
|
||||
nodes with its data cleared (or user can just remove the StorageID
|
||||
value in "VERSION" file under the data directory of the datanode,
|
||||
but this is might not work if VERSION file format has changed
|
||||
*/
|
||||
NameNode.stateChangeLog.info("BLOCK* registerDatanode: " + nodeS
|
||||
+ " is replaced by " + nodeReg + " with the same storageID "
|
||||
+ nodeReg.getStorageID());
|
||||
}
|
||||
// update cluster map
|
||||
getNetworkTopology().remove(nodeS);
|
||||
nodeS.updateRegInfo(nodeReg);
|
||||
nodeS.setDisallowed(false); // Node is in the include list
|
||||
|
||||
// resolve network location
|
||||
resolveNetworkLocation(nodeS);
|
||||
getNetworkTopology().add(nodeS);
|
||||
|
||||
// also treat the registration message as a heartbeat
|
||||
heartbeatManager.register(nodeS);
|
||||
checkDecommissioning(nodeS);
|
||||
return;
|
||||
}
|
||||
|
||||
// this is a new datanode serving a new data storage
|
||||
if ("".equals(nodeReg.getStorageID())) {
|
||||
// this data storage has never been registered
|
||||
// it is either empty or was created by pre-storageID version of DFS
|
||||
nodeReg.setStorageID(newStorageID());
|
||||
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug(
|
||||
"BLOCK* NameSystem.registerDatanode: "
|
||||
+ "new storageID " + nodeReg.getStorageID() + " assigned.");
|
||||
}
|
||||
}
|
||||
// register new datanode
|
||||
DatanodeDescriptor nodeDescr
|
||||
= new DatanodeDescriptor(nodeReg, NetworkTopology.DEFAULT_RACK);
|
||||
resolveNetworkLocation(nodeDescr);
|
||||
addDatanode(nodeDescr);
|
||||
checkDecommissioning(nodeDescr);
|
||||
|
||||
// also treat the registration message as a heartbeat
|
||||
// no need to update its timestamp
|
||||
// because its is done when the descriptor is created
|
||||
heartbeatManager.addDatanode(nodeDescr);
|
||||
try {
|
||||
nodeReg.setExportedKeys(blockManager.getBlockKeys());
|
||||
|
||||
// Checks if the node is not on the hosts list. If it is not, then
|
||||
// it will be disallowed from registering.
|
||||
if (!inHostsList(nodeReg)) {
|
||||
throw new DisallowedDatanodeException(nodeReg);
|
||||
}
|
||||
|
||||
NameNode.stateChangeLog.info("BLOCK* registerDatanode: from "
|
||||
+ nodeReg + " storage " + nodeReg.getStorageID());
|
||||
|
||||
DatanodeDescriptor nodeS = datanodeMap.get(nodeReg.getStorageID());
|
||||
DatanodeDescriptor nodeN = host2DatanodeMap.getDatanodeByXferAddr(
|
||||
nodeReg.getIpAddr(), nodeReg.getXferPort());
|
||||
|
||||
if (nodeN != null && nodeN != nodeS) {
|
||||
NameNode.LOG.info("BLOCK* registerDatanode: " + nodeN);
|
||||
// nodeN previously served a different data storage,
|
||||
// which is not served by anybody anymore.
|
||||
removeDatanode(nodeN);
|
||||
// physically remove node from datanodeMap
|
||||
wipeDatanode(nodeN);
|
||||
nodeN = null;
|
||||
}
|
||||
|
||||
if (nodeS != null) {
|
||||
if (nodeN == nodeS) {
|
||||
// The same datanode has been just restarted to serve the same data
|
||||
// storage. We do not need to remove old data blocks, the delta will
|
||||
// be calculated on the next block report from the datanode
|
||||
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug("BLOCK* registerDatanode: "
|
||||
+ "node restarted.");
|
||||
}
|
||||
} else {
|
||||
// nodeS is found
|
||||
/* The registering datanode is a replacement node for the existing
|
||||
data storage, which from now on will be served by a new node.
|
||||
If this message repeats, both nodes might have same storageID
|
||||
by (insanely rare) random chance. User needs to restart one of the
|
||||
nodes with its data cleared (or user can just remove the StorageID
|
||||
value in "VERSION" file under the data directory of the datanode,
|
||||
but this is might not work if VERSION file format has changed
|
||||
*/
|
||||
NameNode.stateChangeLog.info("BLOCK* registerDatanode: " + nodeS
|
||||
+ " is replaced by " + nodeReg + " with the same storageID "
|
||||
+ nodeReg.getStorageID());
|
||||
}
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
// update cluster map
|
||||
getNetworkTopology().remove(nodeS);
|
||||
nodeS.updateRegInfo(nodeReg);
|
||||
nodeS.setDisallowed(false); // Node is in the include list
|
||||
|
||||
// resolve network location
|
||||
resolveNetworkLocation(nodeS);
|
||||
getNetworkTopology().add(nodeS);
|
||||
|
||||
// also treat the registration message as a heartbeat
|
||||
heartbeatManager.register(nodeS);
|
||||
checkDecommissioning(nodeS);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
removeDatanode(nodeS);
|
||||
wipeDatanode(nodeS);
|
||||
}
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
// this is a new datanode serving a new data storage
|
||||
if ("".equals(nodeReg.getStorageID())) {
|
||||
// this data storage has never been registered
|
||||
// it is either empty or was created by pre-storageID version of DFS
|
||||
nodeReg.setStorageID(newStorageID());
|
||||
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug(
|
||||
"BLOCK* NameSystem.registerDatanode: "
|
||||
+ "new storageID " + nodeReg.getStorageID() + " assigned.");
|
||||
}
|
||||
}
|
||||
|
||||
DatanodeDescriptor nodeDescr
|
||||
= new DatanodeDescriptor(nodeReg, NetworkTopology.DEFAULT_RACK);
|
||||
boolean success = false;
|
||||
try {
|
||||
resolveNetworkLocation(nodeDescr);
|
||||
networktopology.add(nodeDescr);
|
||||
|
||||
// register new datanode
|
||||
addDatanode(nodeDescr);
|
||||
checkDecommissioning(nodeDescr);
|
||||
|
||||
// also treat the registration message as a heartbeat
|
||||
// no need to update its timestamp
|
||||
// because its is done when the descriptor is created
|
||||
heartbeatManager.addDatanode(nodeDescr);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
removeDatanode(nodeDescr);
|
||||
wipeDatanode(nodeDescr);
|
||||
}
|
||||
}
|
||||
} catch (InvalidTopologyException e) {
|
||||
// If the network location is invalid, clear the cached mappings
|
||||
// so that we have a chance to re-add this DataNode with the
|
||||
// correct network location later.
|
||||
dnsToSwitchMapping.reloadCachedMappings();
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -33,6 +33,7 @@ import java.util.Properties;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
||||
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
|
||||
|
@ -663,7 +664,9 @@ public abstract class Storage extends StorageInfo {
|
|||
file.write(jvmName.getBytes(Charsets.UTF_8));
|
||||
LOG.info("Lock on " + lockF + " acquired by nodename " + jvmName);
|
||||
} catch(OverlappingFileLockException oe) {
|
||||
LOG.error("It appears that another namenode " + file.readLine()
|
||||
// Cannot read from the locked file on Windows.
|
||||
String lockingJvmName = Path.WINDOWS ? "" : (" " + file.readLine());
|
||||
LOG.error("It appears that another namenode" + lockingJvmName
|
||||
+ " has already locked the storage directory");
|
||||
file.close();
|
||||
return null;
|
||||
|
|
|
@ -538,7 +538,7 @@ class BPOfferService {
|
|||
// using global fsdataset
|
||||
dn.getFSDataset().invalidate(bcmd.getBlockPoolId(), toDelete);
|
||||
} catch(IOException e) {
|
||||
dn.checkDiskError();
|
||||
// Exceptions caught here are not expected to be disk-related.
|
||||
throw e;
|
||||
}
|
||||
dn.metrics.incrBlocksRemoved(toDelete.length);
|
||||
|
|
|
@ -602,13 +602,13 @@ class BlockReceiver implements Closeable {
|
|||
offsetInBlock > lastCacheDropOffset + CACHE_DROP_LAG_BYTES) {
|
||||
long twoWindowsAgo = lastCacheDropOffset - CACHE_DROP_LAG_BYTES;
|
||||
if (twoWindowsAgo > 0 && dropCacheBehindWrites) {
|
||||
NativeIO.posixFadviseIfPossible(outFd, 0, lastCacheDropOffset,
|
||||
NativeIO.POSIX_FADV_DONTNEED);
|
||||
NativeIO.POSIX.posixFadviseIfPossible(outFd, 0, lastCacheDropOffset,
|
||||
NativeIO.POSIX.POSIX_FADV_DONTNEED);
|
||||
}
|
||||
|
||||
if (syncBehindWrites) {
|
||||
NativeIO.syncFileRangeIfPossible(outFd, lastCacheDropOffset, CACHE_DROP_LAG_BYTES,
|
||||
NativeIO.SYNC_FILE_RANGE_WRITE);
|
||||
NativeIO.POSIX.syncFileRangeIfPossible(outFd, lastCacheDropOffset, CACHE_DROP_LAG_BYTES,
|
||||
NativeIO.POSIX.SYNC_FILE_RANGE_WRITE);
|
||||
}
|
||||
|
||||
lastCacheDropOffset += CACHE_DROP_LAG_BYTES;
|
||||
|
|
|
@ -338,9 +338,9 @@ class BlockSender implements java.io.Closeable {
|
|||
if (blockInFd != null && shouldDropCacheBehindRead && isLongRead()) {
|
||||
// drop the last few MB of the file from cache
|
||||
try {
|
||||
NativeIO.posixFadviseIfPossible(
|
||||
NativeIO.POSIX.posixFadviseIfPossible(
|
||||
blockInFd, lastCacheDropOffset, offset - lastCacheDropOffset,
|
||||
NativeIO.POSIX_FADV_DONTNEED);
|
||||
NativeIO.POSIX.POSIX_FADV_DONTNEED);
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Unable to drop cache on file close", e);
|
||||
}
|
||||
|
@ -637,7 +637,8 @@ class BlockSender implements java.io.Closeable {
|
|||
|
||||
if (isLongRead() && blockInFd != null) {
|
||||
// Advise that this file descriptor will be accessed sequentially.
|
||||
NativeIO.posixFadviseIfPossible(blockInFd, 0, 0, NativeIO.POSIX_FADV_SEQUENTIAL);
|
||||
NativeIO.POSIX.posixFadviseIfPossible(
|
||||
blockInFd, 0, 0, NativeIO.POSIX.POSIX_FADV_SEQUENTIAL);
|
||||
}
|
||||
|
||||
// Trigger readahead of beginning of file if configured.
|
||||
|
@ -725,9 +726,9 @@ class BlockSender implements java.io.Closeable {
|
|||
offset >= nextCacheDropOffset) {
|
||||
long dropLength = offset - lastCacheDropOffset;
|
||||
if (dropLength >= 1024) {
|
||||
NativeIO.posixFadviseIfPossible(blockInFd,
|
||||
NativeIO.POSIX.posixFadviseIfPossible(blockInFd,
|
||||
lastCacheDropOffset, dropLength,
|
||||
NativeIO.POSIX_FADV_DONTNEED);
|
||||
NativeIO.POSIX.POSIX_FADV_DONTNEED);
|
||||
}
|
||||
lastCacheDropOffset += CACHE_DROP_INTERVAL_BYTES;
|
||||
}
|
||||
|
|
|
@ -60,8 +60,11 @@ import java.io.OutputStream;
|
|||
import java.io.PrintStream;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.Socket;
|
||||
import java.net.SocketException;
|
||||
import java.net.SocketTimeoutException;
|
||||
import java.net.URI;
|
||||
import java.net.UnknownHostException;
|
||||
import java.nio.channels.ClosedByInterruptException;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.AbstractList;
|
||||
|
@ -1281,7 +1284,13 @@ public class DataNode extends Configured
|
|||
protected void checkDiskError(Exception e ) throws IOException {
|
||||
|
||||
LOG.warn("checkDiskError: exception: ", e);
|
||||
|
||||
if (e instanceof SocketException || e instanceof SocketTimeoutException
|
||||
|| e instanceof ClosedByInterruptException
|
||||
|| e.getMessage().startsWith("Broken pipe")) {
|
||||
LOG.info("Not checking disk as checkDiskError was called on a network" +
|
||||
" related exception");
|
||||
return;
|
||||
}
|
||||
if (e.getMessage() != null &&
|
||||
e.getMessage().startsWith("No space left on device")) {
|
||||
throw new DiskOutOfSpaceException("No space left on device");
|
||||
|
@ -1593,8 +1602,12 @@ public class DataNode extends Configured
|
|||
} catch (IOException ie) {
|
||||
LOG.warn(bpReg + ":Failed to transfer " + b + " to " +
|
||||
targets[0] + " got ", ie);
|
||||
// check if there are any disk problem
|
||||
checkDiskError();
|
||||
// check if there are any disk problem
|
||||
try{
|
||||
checkDiskError(ie);
|
||||
} catch(IOException e) {
|
||||
LOG.warn("DataNode.checkDiskError failed in run() with: ", e);
|
||||
}
|
||||
|
||||
} finally {
|
||||
xmitsInProgress.getAndDecrement();
|
||||
|
@ -1729,6 +1742,21 @@ public class DataNode extends Configured
|
|||
}
|
||||
}
|
||||
|
||||
// Small wrapper around the DiskChecker class that provides means to mock
|
||||
// DiskChecker static methods and unittest DataNode#getDataDirsFromURIs.
|
||||
static class DataNodeDiskChecker {
|
||||
private FsPermission expectedPermission;
|
||||
|
||||
public DataNodeDiskChecker(FsPermission expectedPermission) {
|
||||
this.expectedPermission = expectedPermission;
|
||||
}
|
||||
|
||||
public void checkDir(LocalFileSystem localFS, Path path)
|
||||
throws DiskErrorException, IOException {
|
||||
DiskChecker.checkDir(localFS, path, expectedPermission);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Make an instance of DataNode after ensuring that at least one of the
|
||||
* given data directories (and their parent directories, if necessary)
|
||||
|
@ -1747,7 +1775,10 @@ public class DataNode extends Configured
|
|||
FsPermission permission = new FsPermission(
|
||||
conf.get(DFS_DATANODE_DATA_DIR_PERMISSION_KEY,
|
||||
DFS_DATANODE_DATA_DIR_PERMISSION_DEFAULT));
|
||||
ArrayList<File> dirs = getDataDirsFromURIs(dataDirs, localFS, permission);
|
||||
DataNodeDiskChecker dataNodeDiskChecker =
|
||||
new DataNodeDiskChecker(permission);
|
||||
ArrayList<File> dirs =
|
||||
getDataDirsFromURIs(dataDirs, localFS, dataNodeDiskChecker);
|
||||
DefaultMetricsSystem.initialize("DataNode");
|
||||
|
||||
assert dirs.size() > 0 : "number of data directories should be > 0";
|
||||
|
@ -1756,7 +1787,8 @@ public class DataNode extends Configured
|
|||
|
||||
// DataNode ctor expects AbstractList instead of List or Collection...
|
||||
static ArrayList<File> getDataDirsFromURIs(Collection<URI> dataDirs,
|
||||
LocalFileSystem localFS, FsPermission permission) throws IOException {
|
||||
LocalFileSystem localFS, DataNodeDiskChecker dataNodeDiskChecker)
|
||||
throws IOException {
|
||||
ArrayList<File> dirs = new ArrayList<File>();
|
||||
StringBuilder invalidDirs = new StringBuilder();
|
||||
for (URI dirURI : dataDirs) {
|
||||
|
@ -1768,7 +1800,7 @@ public class DataNode extends Configured
|
|||
// drop any (illegal) authority in the URI for backwards compatibility
|
||||
File dir = new File(dirURI.getPath());
|
||||
try {
|
||||
DiskChecker.checkDir(localFS, new Path(dir.toURI()), permission);
|
||||
dataNodeDiskChecker.checkDir(localFS, new Path(dir.toURI()));
|
||||
dirs.add(dir);
|
||||
} catch (IOException ioe) {
|
||||
LOG.warn("Invalid " + DFS_DATANODE_DATA_DIR_KEY + " "
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
|
@ -92,6 +93,15 @@ import org.apache.hadoop.util.Time;
|
|||
@InterfaceAudience.Private
|
||||
class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
||||
static final Log LOG = LogFactory.getLog(FsDatasetImpl.class);
|
||||
private final static boolean isNativeIOAvailable;
|
||||
static {
|
||||
isNativeIOAvailable = NativeIO.isAvailable();
|
||||
if (Path.WINDOWS && !isNativeIOAvailable) {
|
||||
LOG.warn("Data node cannot fully support concurrent reading"
|
||||
+ " and writing without native code extensions on Windows.");
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override // FsDatasetSpi
|
||||
public List<FsVolumeImpl> getVolumes() {
|
||||
|
@ -149,6 +159,11 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|||
if (meta == null || !meta.exists()) {
|
||||
return null;
|
||||
}
|
||||
if (isNativeIOAvailable) {
|
||||
return new LengthInputStream(
|
||||
NativeIO.getShareDeleteFileInputStream(meta),
|
||||
meta.length());
|
||||
}
|
||||
return new LengthInputStream(new FileInputStream(meta), meta.length());
|
||||
}
|
||||
|
||||
|
@ -324,18 +339,22 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|||
public InputStream getBlockInputStream(ExtendedBlock b,
|
||||
long seekOffset) throws IOException {
|
||||
File blockFile = getBlockFileNoExistsCheck(b);
|
||||
RandomAccessFile blockInFile;
|
||||
try {
|
||||
blockInFile = new RandomAccessFile(blockFile, "r");
|
||||
} catch (FileNotFoundException fnfe) {
|
||||
throw new IOException("Block " + b + " is not valid. " +
|
||||
"Expected block file at " + blockFile + " does not exist.");
|
||||
}
|
||||
if (isNativeIOAvailable) {
|
||||
return NativeIO.getShareDeleteFileInputStream(blockFile, seekOffset);
|
||||
} else {
|
||||
RandomAccessFile blockInFile;
|
||||
try {
|
||||
blockInFile = new RandomAccessFile(blockFile, "r");
|
||||
} catch (FileNotFoundException fnfe) {
|
||||
throw new IOException("Block " + b + " is not valid. " +
|
||||
"Expected block file at " + blockFile + " does not exist.");
|
||||
}
|
||||
|
||||
if (seekOffset > 0) {
|
||||
blockInFile.seek(seekOffset);
|
||||
if (seekOffset > 0) {
|
||||
blockInFile.seek(seekOffset);
|
||||
}
|
||||
return new FileInputStream(blockInFile.getFD());
|
||||
}
|
||||
return new FileInputStream(blockInFile.getFD());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -39,6 +39,8 @@ public class CheckpointConf {
|
|||
/** checkpoint once every this many transactions, regardless of time */
|
||||
private final long checkpointTxnCount;
|
||||
|
||||
/** maxium number of retries when merge errors occur */
|
||||
private final int maxRetriesOnMergeError;
|
||||
|
||||
public CheckpointConf(Configuration conf) {
|
||||
checkpointCheckPeriod = conf.getLong(
|
||||
|
@ -49,6 +51,8 @@ public class CheckpointConf {
|
|||
DFS_NAMENODE_CHECKPOINT_PERIOD_DEFAULT);
|
||||
checkpointTxnCount = conf.getLong(DFS_NAMENODE_CHECKPOINT_TXNS_KEY,
|
||||
DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT);
|
||||
maxRetriesOnMergeError = conf.getInt(DFS_NAMENODE_CHECKPOINT_MAX_RETRIES_KEY,
|
||||
DFS_NAMENODE_CHECKPOINT_MAX_RETRIES_DEFAULT);
|
||||
warnForDeprecatedConfigs(conf);
|
||||
}
|
||||
|
||||
|
@ -75,4 +79,8 @@ public class CheckpointConf {
|
|||
public long getTxnCount() {
|
||||
return checkpointTxnCount;
|
||||
}
|
||||
|
||||
public int getMaxRetriesOnMergeError() {
|
||||
return maxRetriesOnMergeError;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,6 +33,7 @@ class CheckpointFaultInjector {
|
|||
|
||||
public void beforeGetImageSetsHeaders() throws IOException {}
|
||||
public void afterSecondaryCallsRollEditLog() throws IOException {}
|
||||
public void duringMerge() throws IOException {}
|
||||
public void afterSecondaryUploadsNewImage() throws IOException {}
|
||||
public void aboutToSendFile(File localfile) throws IOException {}
|
||||
|
||||
|
@ -43,4 +44,5 @@ class CheckpointFaultInjector {
|
|||
return false;
|
||||
}
|
||||
|
||||
public void afterMD5Rename() throws IOException {}
|
||||
}
|
||||
|
|
|
@ -142,4 +142,9 @@ class EditLogBackupInputStream extends EditLogInputStream {
|
|||
public boolean isInProgress() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setMaxOpSize(int maxOpSize) {
|
||||
reader.setMaxOpSize(maxOpSize);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,6 +32,7 @@ import java.security.PrivilegedExceptionAction;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.server.common.Storage;
|
||||
import org.apache.hadoop.hdfs.server.namenode.TransferFsImage.HttpGetFailedException;
|
||||
|
@ -53,6 +54,7 @@ public class EditLogFileInputStream extends EditLogInputStream {
|
|||
private final long firstTxId;
|
||||
private final long lastTxId;
|
||||
private final boolean isInProgress;
|
||||
private int maxOpSize;
|
||||
static private enum State {
|
||||
UNINIT,
|
||||
OPEN,
|
||||
|
@ -118,6 +120,7 @@ public class EditLogFileInputStream extends EditLogInputStream {
|
|||
this.firstTxId = firstTxId;
|
||||
this.lastTxId = lastTxId;
|
||||
this.isInProgress = isInProgress;
|
||||
this.maxOpSize = DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_DEFAULT;
|
||||
}
|
||||
|
||||
private void init() throws LogHeaderCorruptException, IOException {
|
||||
|
@ -134,6 +137,7 @@ public class EditLogFileInputStream extends EditLogInputStream {
|
|||
throw new LogHeaderCorruptException("No header found in log");
|
||||
}
|
||||
reader = new FSEditLogOp.Reader(dataIn, tracker, logVersion);
|
||||
reader.setMaxOpSize(maxOpSize);
|
||||
state = State.OPEN;
|
||||
} finally {
|
||||
if (reader == null) {
|
||||
|
@ -412,5 +416,12 @@ public class EditLogFileInputStream extends EditLogInputStream {
|
|||
return url.toString();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void setMaxOpSize(int maxOpSize) {
|
||||
this.maxOpSize = maxOpSize;
|
||||
if (reader != null) {
|
||||
reader.setMaxOpSize(maxOpSize);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -165,4 +165,9 @@ public abstract class EditLogInputStream implements Closeable {
|
|||
* Return true if this stream is in progress, false if it is finalized.
|
||||
*/
|
||||
public abstract boolean isInProgress();
|
||||
|
||||
/**
|
||||
* Set the maximum opcode size in bytes.
|
||||
*/
|
||||
public abstract void setMaxOpSize(int maxOpSize);
|
||||
}
|
||||
|
|
|
@ -1343,6 +1343,11 @@ public class FSDirectory implements Closeable {
|
|||
|
||||
// fill up the inodes in the path from this inode to root
|
||||
for (int i = 0; i < depth; i++) {
|
||||
if (inode == null) {
|
||||
NameNode.stateChangeLog.warn("Could not get full path."
|
||||
+ " Corresponding file might have deleted already.");
|
||||
return null;
|
||||
}
|
||||
inodes[depth-i-1] = inode;
|
||||
inode = inode.parent;
|
||||
}
|
||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.hadoop.io.WritableFactory;
|
|||
import org.apache.hadoop.hdfs.util.XMLUtils;
|
||||
import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
|
||||
import org.apache.hadoop.hdfs.util.XMLUtils.Stanza;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DeprecatedUTF8;
|
||||
import org.xml.sax.ContentHandler;
|
||||
import org.xml.sax.SAXException;
|
||||
|
@ -75,11 +76,6 @@ import java.io.EOFException;
|
|||
public abstract class FSEditLogOp {
|
||||
public final FSEditLogOpCodes opCode;
|
||||
long txid;
|
||||
/**
|
||||
* Opcode size is limited to 1.5 megabytes
|
||||
*/
|
||||
public static final int MAX_OP_SIZE = (3 * 1024 * 1024) / 2;
|
||||
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
final public static class OpInstanceCache {
|
||||
|
@ -2246,6 +2242,7 @@ public abstract class FSEditLogOp {
|
|||
private final int logVersion;
|
||||
private final Checksum checksum;
|
||||
private final OpInstanceCache cache;
|
||||
private int maxOpSize;
|
||||
|
||||
/**
|
||||
* Construct the reader
|
||||
|
@ -2253,7 +2250,8 @@ public abstract class FSEditLogOp {
|
|||
* @param logVersion The version of the data coming from the stream.
|
||||
*/
|
||||
@SuppressWarnings("deprecation")
|
||||
public Reader(DataInputStream in, StreamLimiter limiter, int logVersion) {
|
||||
public Reader(DataInputStream in, StreamLimiter limiter,
|
||||
int logVersion) {
|
||||
this.logVersion = logVersion;
|
||||
if (LayoutVersion.supports(Feature.EDITS_CHESKUM, logVersion)) {
|
||||
this.checksum = new PureJavaCrc32();
|
||||
|
@ -2269,6 +2267,11 @@ public abstract class FSEditLogOp {
|
|||
}
|
||||
this.limiter = limiter;
|
||||
this.cache = new OpInstanceCache();
|
||||
this.maxOpSize = DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_DEFAULT;
|
||||
}
|
||||
|
||||
public void setMaxOpSize(int maxOpSize) {
|
||||
this.maxOpSize = maxOpSize;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2363,8 +2366,8 @@ public abstract class FSEditLogOp {
|
|||
* problematic byte. This usually means the beginning of the opcode.
|
||||
*/
|
||||
private FSEditLogOp decodeOp() throws IOException {
|
||||
limiter.setLimit(MAX_OP_SIZE);
|
||||
in.mark(MAX_OP_SIZE);
|
||||
limiter.setLimit(maxOpSize);
|
||||
in.mark(maxOpSize);
|
||||
|
||||
if (checksum != null) {
|
||||
checksum.reset();
|
||||
|
|
|
@ -49,6 +49,7 @@ import static org.apache.hadoop.util.Time.now;
|
|||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
||||
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.FSImageFile;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
|
||||
import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
|
||||
|
@ -584,11 +585,11 @@ public class FSImage implements Closeable {
|
|||
boolean loadFSImage(FSNamesystem target, MetaRecoveryContext recovery)
|
||||
throws IOException {
|
||||
FSImageStorageInspector inspector = storage.readAndInspectDirs();
|
||||
FSImageFile imageFile = null;
|
||||
|
||||
isUpgradeFinalized = inspector.isUpgradeFinalized();
|
||||
|
||||
FSImageStorageInspector.FSImageFile imageFile
|
||||
= inspector.getLatestImage();
|
||||
List<FSImageFile> imageFiles = inspector.getLatestImages();
|
||||
boolean needToSave = inspector.needToSave();
|
||||
|
||||
Iterable<EditLogInputStream> editStreams = null;
|
||||
|
@ -601,14 +602,20 @@ public class FSImage implements Closeable {
|
|||
// we better be able to load all the edits. If we're the standby NN, it's
|
||||
// OK to not be able to read all of edits right now.
|
||||
long toAtLeastTxId = editLog.isOpenForWrite() ? inspector.getMaxSeenTxId() : 0;
|
||||
editStreams = editLog.selectInputStreams(imageFile.getCheckpointTxId() + 1,
|
||||
editStreams = editLog.selectInputStreams(
|
||||
imageFiles.get(0).getCheckpointTxId() + 1,
|
||||
toAtLeastTxId, recovery, false);
|
||||
} else {
|
||||
editStreams = FSImagePreTransactionalStorageInspector
|
||||
.getEditLogStreams(storage);
|
||||
}
|
||||
int maxOpSize = conf.getInt(DFSConfigKeys.
|
||||
DFS_NAMENODE_MAX_OP_SIZE_KEY,
|
||||
DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_DEFAULT);
|
||||
for (EditLogInputStream elis : editStreams) {
|
||||
elis.setMaxOpSize(maxOpSize);
|
||||
}
|
||||
|
||||
LOG.debug("Planning to load image :\n" + imageFile);
|
||||
for (EditLogInputStream l : editStreams) {
|
||||
LOG.debug("Planning to load edit log stream: " + l);
|
||||
}
|
||||
|
@ -616,34 +623,21 @@ public class FSImage implements Closeable {
|
|||
LOG.info("No edit log streams selected.");
|
||||
}
|
||||
|
||||
try {
|
||||
StorageDirectory sdForProperties = imageFile.sd;
|
||||
storage.readProperties(sdForProperties);
|
||||
|
||||
if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT,
|
||||
getLayoutVersion())) {
|
||||
// For txid-based layout, we should have a .md5 file
|
||||
// next to the image file
|
||||
loadFSImage(imageFile.getFile(), target, recovery);
|
||||
} else if (LayoutVersion.supports(Feature.FSIMAGE_CHECKSUM,
|
||||
getLayoutVersion())) {
|
||||
// In 0.22, we have the checksum stored in the VERSION file.
|
||||
String md5 = storage.getDeprecatedProperty(
|
||||
NNStorage.DEPRECATED_MESSAGE_DIGEST_PROPERTY);
|
||||
if (md5 == null) {
|
||||
throw new InconsistentFSStateException(sdForProperties.getRoot(),
|
||||
"Message digest property " +
|
||||
NNStorage.DEPRECATED_MESSAGE_DIGEST_PROPERTY +
|
||||
" not set for storage directory " + sdForProperties.getRoot());
|
||||
}
|
||||
loadFSImage(imageFile.getFile(), new MD5Hash(md5), target, recovery);
|
||||
} else {
|
||||
// We don't have any record of the md5sum
|
||||
loadFSImage(imageFile.getFile(), null, target, recovery);
|
||||
for (int i = 0; i < imageFiles.size(); i++) {
|
||||
try {
|
||||
imageFile = imageFiles.get(i);
|
||||
loadFSImageFile(target, recovery, imageFile);
|
||||
break;
|
||||
} catch (IOException ioe) {
|
||||
LOG.error("Failed to load image from " + imageFile, ioe);
|
||||
target.clear();
|
||||
imageFile = null;
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
}
|
||||
// Failed to load any images, error out
|
||||
if (imageFile == null) {
|
||||
FSEditLog.closeAllStreams(editStreams);
|
||||
throw new IOException("Failed to load image from " + imageFile, ioe);
|
||||
throw new IOException("Failed to load an FSImage file!");
|
||||
}
|
||||
long txnsAdvanced = loadEdits(editStreams, target, recovery);
|
||||
needToSave |= needsResaveBasedOnStaleCheckpoint(imageFile.getFile(),
|
||||
|
@ -652,6 +646,35 @@ public class FSImage implements Closeable {
|
|||
return needToSave;
|
||||
}
|
||||
|
||||
void loadFSImageFile(FSNamesystem target, MetaRecoveryContext recovery,
|
||||
FSImageFile imageFile) throws IOException {
|
||||
LOG.debug("Planning to load image :\n" + imageFile);
|
||||
StorageDirectory sdForProperties = imageFile.sd;
|
||||
storage.readProperties(sdForProperties);
|
||||
|
||||
if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT,
|
||||
getLayoutVersion())) {
|
||||
// For txid-based layout, we should have a .md5 file
|
||||
// next to the image file
|
||||
loadFSImage(imageFile.getFile(), target, recovery);
|
||||
} else if (LayoutVersion.supports(Feature.FSIMAGE_CHECKSUM,
|
||||
getLayoutVersion())) {
|
||||
// In 0.22, we have the checksum stored in the VERSION file.
|
||||
String md5 = storage.getDeprecatedProperty(
|
||||
NNStorage.DEPRECATED_MESSAGE_DIGEST_PROPERTY);
|
||||
if (md5 == null) {
|
||||
throw new InconsistentFSStateException(sdForProperties.getRoot(),
|
||||
"Message digest property " +
|
||||
NNStorage.DEPRECATED_MESSAGE_DIGEST_PROPERTY +
|
||||
" not set for storage directory " + sdForProperties.getRoot());
|
||||
}
|
||||
loadFSImage(imageFile.getFile(), new MD5Hash(md5), target, recovery);
|
||||
} else {
|
||||
// We don't have any record of the md5sum
|
||||
loadFSImage(imageFile.getFile(), null, target, recovery);
|
||||
}
|
||||
}
|
||||
|
||||
public void initEditLog() {
|
||||
Preconditions.checkState(getNamespaceID() != 0,
|
||||
"Must know namespace ID before initting edit log");
|
||||
|
@ -1080,7 +1103,7 @@ public class FSImage implements Closeable {
|
|||
*/
|
||||
public synchronized void saveDigestAndRenameCheckpointImage(
|
||||
long txid, MD5Hash digest) throws IOException {
|
||||
renameCheckpoint(txid);
|
||||
// Write and rename MD5 file
|
||||
List<StorageDirectory> badSds = Lists.newArrayList();
|
||||
|
||||
for (StorageDirectory sd : storage.dirIterable(NameNodeDirType.IMAGE)) {
|
||||
|
@ -1093,6 +1116,10 @@ public class FSImage implements Closeable {
|
|||
}
|
||||
storage.reportErrorsOnDirectories(badSds);
|
||||
|
||||
CheckpointFaultInjector.getInstance().afterMD5Rename();
|
||||
|
||||
// Rename image from tmp file
|
||||
renameCheckpoint(txid);
|
||||
// So long as this is the newest image available,
|
||||
// advertise it as such to other checkpointers
|
||||
// from now on
|
||||
|
|
|
@ -232,8 +232,8 @@ class FSImageFormat {
|
|||
loadSecretManagerState(in);
|
||||
|
||||
// make sure to read to the end of file
|
||||
int eof = in.read();
|
||||
assert eof == -1 : "Should have reached the end of image file " + curFile;
|
||||
boolean eof = (in.read() == -1);
|
||||
assert eof : "Should have reached the end of image file " + curFile;
|
||||
} finally {
|
||||
in.close();
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
|
@ -146,7 +147,7 @@ class FSImagePreTransactionalStorageInspector extends FSImageStorageInspector {
|
|||
}
|
||||
|
||||
@Override
|
||||
FSImageFile getLatestImage() throws IOException {
|
||||
List<FSImageFile> getLatestImages() throws IOException {
|
||||
// We should have at least one image and one edits dirs
|
||||
if (latestNameSD == null)
|
||||
throw new IOException("Image file is not found in " + imageDirs);
|
||||
|
@ -176,9 +177,12 @@ class FSImagePreTransactionalStorageInspector extends FSImageStorageInspector {
|
|||
|
||||
needToSaveAfterRecovery = doRecovery();
|
||||
|
||||
return new FSImageFile(latestNameSD,
|
||||
FSImageFile file = new FSImageFile(latestNameSD,
|
||||
NNStorage.getStorageFile(latestNameSD, NameNodeFile.IMAGE),
|
||||
HdfsConstants.INVALID_TXID);
|
||||
LinkedList<FSImageFile> ret = new LinkedList<FSImageFile>();
|
||||
ret.add(file);
|
||||
return ret;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,6 +19,8 @@ package org.apache.hadoop.hdfs.server.namenode;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
@ -45,7 +47,7 @@ abstract class FSImageStorageInspector {
|
|||
* Get the image files which should be loaded into the filesystem.
|
||||
* @throws IOException if not enough files are available (eg no image found in any directory)
|
||||
*/
|
||||
abstract FSImageFile getLatestImage() throws IOException;
|
||||
abstract List<FSImageFile> getLatestImages() throws IOException;
|
||||
|
||||
/**
|
||||
* Get the minimum tx id which should be loaded with this set of images.
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.io.File;
|
|||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
@ -108,24 +109,31 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return the image that has the most recent associated transaction ID.
|
||||
* If there are multiple storage directories which contain equal images
|
||||
* the storage directory that was inspected first will be preferred.
|
||||
* @return the image files that have the most recent associated
|
||||
* transaction IDs. If there are multiple storage directories which
|
||||
* contain equal images, we'll return them all.
|
||||
*
|
||||
* @throws FileNotFoundException if not images are found.
|
||||
*/
|
||||
@Override
|
||||
FSImageFile getLatestImage() throws IOException {
|
||||
if (foundImages.isEmpty()) {
|
||||
throw new FileNotFoundException("No valid image files found");
|
||||
}
|
||||
|
||||
FSImageFile ret = null;
|
||||
List<FSImageFile> getLatestImages() throws IOException {
|
||||
LinkedList<FSImageFile> ret = new LinkedList<FSImageFile>();
|
||||
for (FSImageFile img : foundImages) {
|
||||
if (ret == null || img.txId > ret.txId) {
|
||||
ret = img;
|
||||
if (ret.isEmpty()) {
|
||||
ret.add(img);
|
||||
} else {
|
||||
FSImageFile cur = ret.getFirst();
|
||||
if (cur.txId == img.txId) {
|
||||
ret.add(img);
|
||||
} else if (cur.txId < img.txId) {
|
||||
ret.clear();
|
||||
ret.add(img);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (ret.isEmpty()) {
|
||||
throw new FileNotFoundException("No valid image files found");
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
import java.util.Stack;
|
||||
|
@ -31,14 +32,20 @@ import org.apache.hadoop.fs.permission.FsPermission;
|
|||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
|
||||
/** Perform permission checking in {@link FSNamesystem}. */
|
||||
/**
|
||||
* Class that helps in checking file system permission.
|
||||
* The state of this class need not be synchronized as it has data structures that
|
||||
* are read-only.
|
||||
*
|
||||
* Some of the helper methods are gaurded by {@link FSNamesystem#readLock()}.
|
||||
*/
|
||||
class FSPermissionChecker {
|
||||
static final Log LOG = LogFactory.getLog(UserGroupInformation.class);
|
||||
|
||||
private final UserGroupInformation ugi;
|
||||
public final String user;
|
||||
private final Set<String> groups = new HashSet<String>();
|
||||
public final boolean isSuper;
|
||||
private final String user;
|
||||
/** A set with group namess. Not synchronized since it is unmodifiable */
|
||||
private final Set<String> groups;
|
||||
private final boolean isSuper;
|
||||
|
||||
FSPermissionChecker(String fsOwner, String supergroup
|
||||
) throws AccessControlException{
|
||||
|
@ -47,10 +54,9 @@ class FSPermissionChecker {
|
|||
} catch (IOException e) {
|
||||
throw new AccessControlException(e);
|
||||
}
|
||||
|
||||
groups.addAll(Arrays.asList(ugi.getGroupNames()));
|
||||
HashSet<String> s = new HashSet<String>(Arrays.asList(ugi.getGroupNames()));
|
||||
groups = Collections.unmodifiableSet(s);
|
||||
user = ugi.getShortUserName();
|
||||
|
||||
isSuper = user.equals(fsOwner) || groups.contains(supergroup);
|
||||
}
|
||||
|
||||
|
@ -60,20 +66,23 @@ class FSPermissionChecker {
|
|||
*/
|
||||
public boolean containsGroup(String group) {return groups.contains(group);}
|
||||
|
||||
public String getUser() {
|
||||
return user;
|
||||
}
|
||||
|
||||
public boolean isSuperUser() {
|
||||
return isSuper;
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify if the caller has the required permission. This will result into
|
||||
* an exception if the caller is not allowed to access the resource.
|
||||
* @param owner owner of the system
|
||||
* @param supergroup supergroup of the system
|
||||
*/
|
||||
public static void checkSuperuserPrivilege(UserGroupInformation owner,
|
||||
String supergroup)
|
||||
throws AccessControlException {
|
||||
FSPermissionChecker checker =
|
||||
new FSPermissionChecker(owner.getShortUserName(), supergroup);
|
||||
if (!checker.isSuper) {
|
||||
public void checkSuperuserPrivilege()
|
||||
throws AccessControlException {
|
||||
if (!isSuper) {
|
||||
throw new AccessControlException("Access denied for user "
|
||||
+ checker.user + ". Superuser privilege is required");
|
||||
+ user + ". Superuser privilege is required");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -103,9 +112,11 @@ class FSPermissionChecker {
|
|||
* @param subAccess If path is a directory,
|
||||
* it is the access required of the path and all the sub-directories.
|
||||
* If path is not a directory, there is no effect.
|
||||
* @return a PermissionChecker object which caches data for later use.
|
||||
* @throws AccessControlException
|
||||
* @throws UnresolvedLinkException
|
||||
*
|
||||
* Guarded by {@link FSNamesystem#readLock()}
|
||||
* Caller of this method must hold that lock.
|
||||
*/
|
||||
void checkPermission(String path, INodeDirectory root, boolean doCheckOwner,
|
||||
FsAction ancestorAccess, FsAction parentAccess, FsAction access,
|
||||
|
@ -148,6 +159,7 @@ class FSPermissionChecker {
|
|||
}
|
||||
}
|
||||
|
||||
/** Guarded by {@link FSNamesystem#readLock()} */
|
||||
private void checkOwner(INode inode) throws AccessControlException {
|
||||
if (inode != null && user.equals(inode.getUserName())) {
|
||||
return;
|
||||
|
@ -155,6 +167,7 @@ class FSPermissionChecker {
|
|||
throw new AccessControlException("Permission denied");
|
||||
}
|
||||
|
||||
/** Guarded by {@link FSNamesystem#readLock()} */
|
||||
private void checkTraverse(INode[] inodes, int last
|
||||
) throws AccessControlException {
|
||||
for(int j = 0; j <= last; j++) {
|
||||
|
@ -162,6 +175,7 @@ class FSPermissionChecker {
|
|||
}
|
||||
}
|
||||
|
||||
/** Guarded by {@link FSNamesystem#readLock()} */
|
||||
private void checkSubAccess(INode inode, FsAction access
|
||||
) throws AccessControlException {
|
||||
if (inode == null || !inode.isDirectory()) {
|
||||
|
@ -181,11 +195,13 @@ class FSPermissionChecker {
|
|||
}
|
||||
}
|
||||
|
||||
/** Guarded by {@link FSNamesystem#readLock()} */
|
||||
private void check(INode[] inodes, int i, FsAction access
|
||||
) throws AccessControlException {
|
||||
check(i >= 0? inodes[i]: null, access);
|
||||
}
|
||||
|
||||
/** Guarded by {@link FSNamesystem#readLock()} */
|
||||
private void check(INode inode, FsAction access
|
||||
) throws AccessControlException {
|
||||
if (inode == null) {
|
||||
|
@ -206,7 +222,9 @@ class FSPermissionChecker {
|
|||
+ ", access=" + access + ", inode=" + inode);
|
||||
}
|
||||
|
||||
private void checkStickyBit(INode parent, INode inode) throws AccessControlException {
|
||||
/** Guarded by {@link FSNamesystem#readLock()} */
|
||||
private void checkStickyBit(INode parent, INode inode)
|
||||
throws AccessControlException {
|
||||
if(!parent.getFsPermission().getStickyBit()) {
|
||||
return;
|
||||
}
|
||||
|
|
|
@ -282,7 +282,11 @@ abstract class INode implements Comparable<byte[]> {
|
|||
|
||||
String getLocalParentDir() {
|
||||
INode inode = isRoot() ? this : getParent();
|
||||
return (inode != null) ? inode.getFullPathName() : "";
|
||||
String parentDir = "";
|
||||
if (inode != null) {
|
||||
parentDir = inode.getFullPathName();
|
||||
}
|
||||
return (parentDir != null) ? parentDir : "";
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -25,10 +25,10 @@ import java.util.Map;
|
|||
|
||||
import javax.servlet.ServletContext;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
import org.apache.hadoop.hdfs.server.common.JspHelper;
|
||||
import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
|
||||
import org.apache.hadoop.hdfs.web.AuthFilter;
|
||||
|
@ -77,7 +77,8 @@ public class NameNodeHttpServer {
|
|||
if (UserGroupInformation.isSecurityEnabled()) {
|
||||
initSpnego(conf,
|
||||
DFSConfigKeys.DFS_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY,
|
||||
DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY);
|
||||
DFSUtil.getSpnegoKeytabKey(conf,
|
||||
DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY));
|
||||
}
|
||||
if (WebHdfsFileSystem.isEnabled(conf, LOG)) {
|
||||
//add SPNEGO authentication filter for webhdfs
|
||||
|
@ -112,11 +113,8 @@ public class NameNodeHttpServer {
|
|||
DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY +
|
||||
"' is not set.");
|
||||
}
|
||||
String httpKeytab = conf.get(
|
||||
DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY);
|
||||
if (httpKeytab == null) {
|
||||
httpKeytab = conf.get(DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY);
|
||||
}
|
||||
String httpKeytab = conf.get(DFSUtil.getSpnegoKeytabKey(conf,
|
||||
DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY));
|
||||
if (httpKeytab != null && !httpKeytab.isEmpty()) {
|
||||
params.put(
|
||||
DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY,
|
||||
|
|
|
@ -337,7 +337,6 @@ class NameNodeRpcServer implements NamenodeProtocols {
|
|||
throw new IllegalArgumentException(
|
||||
"Unexpected not positive size: "+size);
|
||||
}
|
||||
namesystem.checkOperation(OperationCategory.READ);
|
||||
namesystem.checkSuperuserPrivilege();
|
||||
return namesystem.getBlockManager().getBlocks(datanode, size);
|
||||
}
|
||||
|
@ -707,7 +706,6 @@ class NameNodeRpcServer implements NamenodeProtocols {
|
|||
@Override // ClientProtocol
|
||||
public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
|
||||
throws IOException {
|
||||
namesystem.checkOperation(OperationCategory.UNCHECKED);
|
||||
DatanodeInfo results[] = namesystem.datanodeReport(type);
|
||||
if (results == null ) {
|
||||
throw new IOException("Cannot find datanode report");
|
||||
|
@ -732,19 +730,16 @@ class NameNodeRpcServer implements NamenodeProtocols {
|
|||
|
||||
@Override // ClientProtocol
|
||||
public boolean restoreFailedStorage(String arg) throws IOException {
|
||||
namesystem.checkOperation(OperationCategory.UNCHECKED);
|
||||
return namesystem.restoreFailedStorage(arg);
|
||||
}
|
||||
|
||||
@Override // ClientProtocol
|
||||
public void saveNamespace() throws IOException {
|
||||
namesystem.checkOperation(OperationCategory.UNCHECKED);
|
||||
namesystem.saveNamespace();
|
||||
}
|
||||
|
||||
@Override // ClientProtocol
|
||||
public long rollEdits() throws AccessControlException, IOException {
|
||||
namesystem.checkOperation(OperationCategory.JOURNAL);
|
||||
CheckpointSignature sig = namesystem.rollEditLog();
|
||||
return sig.getCurSegmentTxId();
|
||||
}
|
||||
|
@ -789,7 +784,6 @@ class NameNodeRpcServer implements NamenodeProtocols {
|
|||
|
||||
@Override // ClientProtocol
|
||||
public void metaSave(String filename) throws IOException {
|
||||
namesystem.checkOperation(OperationCategory.UNCHECKED);
|
||||
namesystem.metaSave(filename);
|
||||
}
|
||||
|
||||
|
|
|
@ -18,7 +18,9 @@
|
|||
package org.apache.hadoop.hdfs.server.namenode;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
|
||||
import org.apache.hadoop.hdfs.util.RwLock;
|
||||
import org.apache.hadoop.ipc.StandbyException;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
|
||||
/** Namesystem operations. */
|
||||
|
@ -38,4 +40,6 @@ public interface Namesystem extends RwLock, SafeMode {
|
|||
public boolean isGenStampInFuture(long generationStamp);
|
||||
|
||||
public void adjustSafeModeBlockTotals(int deltaSafe, int deltaTotal);
|
||||
|
||||
public void checkOperation(OperationCategory read) throws StandbyException;
|
||||
}
|
|
@ -267,4 +267,11 @@ class RedundantEditLogInputStream extends EditLogInputStream {
|
|||
super(msg);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setMaxOpSize(int maxOpSize) {
|
||||
for (EditLogInputStream elis : streams) {
|
||||
elis.setMaxOpSize(maxOpSize);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -144,6 +144,11 @@ public class SecondaryNameNode implements Runnable {
|
|||
return checkpointImage;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
int getMergeErrorCount() {
|
||||
return checkpointImage.getMergeErrorCount();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
FSNamesystem getFSNamesystem() {
|
||||
return namesystem;
|
||||
|
@ -250,15 +255,11 @@ public class SecondaryNameNode implements Runnable {
|
|||
new AccessControlList(conf.get(DFS_ADMIN, " "))) {
|
||||
{
|
||||
if (UserGroupInformation.isSecurityEnabled()) {
|
||||
String httpKeytabKey = DFSConfigKeys.
|
||||
DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY;
|
||||
if (null == conf.get(httpKeytabKey)) {
|
||||
httpKeytabKey = DFSConfigKeys.DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY;
|
||||
}
|
||||
initSpnego(
|
||||
conf,
|
||||
DFSConfigKeys.DFS_SECONDARY_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY,
|
||||
httpKeytabKey);
|
||||
DFSUtil.getSpnegoKeytabKey(conf,
|
||||
DFSConfigKeys.DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY));
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -339,6 +340,7 @@ public class SecondaryNameNode implements Runnable {
|
|||
// number of transactions in the edit log that haven't yet been checkpointed.
|
||||
//
|
||||
long period = checkpointConf.getCheckPeriod();
|
||||
int maxRetries = checkpointConf.getMaxRetriesOnMergeError();
|
||||
|
||||
while (shouldRun) {
|
||||
try {
|
||||
|
@ -364,6 +366,13 @@ public class SecondaryNameNode implements Runnable {
|
|||
} catch (IOException e) {
|
||||
LOG.error("Exception in doCheckpoint", e);
|
||||
e.printStackTrace();
|
||||
// Prevent a huge number of edits from being created due to
|
||||
// unrecoverable conditions and endless retries.
|
||||
if (checkpointImage.getMergeErrorCount() > maxRetries) {
|
||||
LOG.fatal("Merging failed " +
|
||||
checkpointImage.getMergeErrorCount() + " times.");
|
||||
terminate(1);
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
LOG.fatal("Throwable Exception in doCheckpoint", e);
|
||||
e.printStackTrace();
|
||||
|
@ -498,9 +507,21 @@ public class SecondaryNameNode implements Runnable {
|
|||
RemoteEditLogManifest manifest =
|
||||
namenode.getEditLogManifest(sig.mostRecentCheckpointTxId + 1);
|
||||
|
||||
// Fetch fsimage and edits. Reload the image if previous merge failed.
|
||||
loadImage |= downloadCheckpointFiles(
|
||||
fsName, checkpointImage, sig, manifest); // Fetch fsimage and edits
|
||||
doMerge(sig, manifest, loadImage, checkpointImage, namesystem);
|
||||
fsName, checkpointImage, sig, manifest) |
|
||||
checkpointImage.hasMergeError();
|
||||
try {
|
||||
doMerge(sig, manifest, loadImage, checkpointImage, namesystem);
|
||||
} catch (IOException ioe) {
|
||||
// A merge error occurred. The in-memory file system state may be
|
||||
// inconsistent, so the image and edits need to be reloaded.
|
||||
checkpointImage.setMergeError();
|
||||
throw ioe;
|
||||
}
|
||||
// Clear any error since merge was successful.
|
||||
checkpointImage.clearMergeError();
|
||||
|
||||
|
||||
//
|
||||
// Upload the new image into the NameNode. Then tell the Namenode
|
||||
|
@ -754,6 +775,7 @@ public class SecondaryNameNode implements Runnable {
|
|||
|
||||
static class CheckpointStorage extends FSImage {
|
||||
|
||||
private int mergeErrorCount;
|
||||
private static class CheckpointLogPurger implements LogsPurgeable {
|
||||
|
||||
private NNStorage storage;
|
||||
|
@ -815,6 +837,7 @@ public class SecondaryNameNode implements Runnable {
|
|||
// we shouldn't have any editLog instance. Setting to null
|
||||
// makes sure we don't accidentally depend on it.
|
||||
editLog = null;
|
||||
mergeErrorCount = 0;
|
||||
|
||||
// Replace the archival manager with one that can actually work on the
|
||||
// 2NN's edits storage.
|
||||
|
@ -881,7 +904,24 @@ public class SecondaryNameNode implements Runnable {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
boolean hasMergeError() {
|
||||
return (mergeErrorCount > 0);
|
||||
}
|
||||
|
||||
int getMergeErrorCount() {
|
||||
return mergeErrorCount;
|
||||
}
|
||||
|
||||
void setMergeError() {
|
||||
mergeErrorCount++;
|
||||
}
|
||||
|
||||
void clearMergeError() {
|
||||
mergeErrorCount = 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensure that the current/ directory exists in all storage
|
||||
* directories
|
||||
|
@ -915,7 +955,9 @@ public class SecondaryNameNode implements Runnable {
|
|||
dstImage.reloadFromImageFile(file, dstNamesystem);
|
||||
dstNamesystem.dir.imageLoadComplete();
|
||||
}
|
||||
|
||||
// error simulation code for junit test
|
||||
CheckpointFaultInjector.getInstance().duringMerge();
|
||||
|
||||
Checkpointer.rollForwardByApplyingLogs(manifest, dstImage, dstNamesystem);
|
||||
// The following has the side effect of purging old fsimages/edit logs.
|
||||
dstImage.saveFSImageInAllDirs(dstNamesystem, dstImage.getLastAppliedTxId());
|
||||
|
|
|
@ -229,7 +229,6 @@ public class TransferFsImage {
|
|||
SecurityUtil.openSecureHttpConnection(url);
|
||||
|
||||
if (timeout <= 0) {
|
||||
// Set the ping interval as timeout
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
timeout = conf.getInt(DFSConfigKeys.DFS_IMAGE_TRANSFER_TIMEOUT_KEY,
|
||||
DFSConfigKeys.DFS_IMAGE_TRANSFER_TIMEOUT_DEFAULT);
|
||||
|
|
|
@ -64,9 +64,17 @@ public interface HAContext {
|
|||
void writeUnlock();
|
||||
|
||||
/**
|
||||
* Verify that the given operation category is allowed in the
|
||||
* current state. This is to allow NN implementations (eg BackupNode)
|
||||
* to override it with node-specific handling.
|
||||
* Verify that the given operation category is allowed in the current state.
|
||||
* This is to allow NN implementations (eg BackupNode) to override it with
|
||||
* node-specific handling.
|
||||
*
|
||||
* If the operation which is being checked will be taking the FSNS lock, it's
|
||||
* advisable to check the operation category both immediately before and after
|
||||
* taking the lock. This is because clients rely on the StandbyException
|
||||
* thrown by this method in order to trigger client failover, and if a client
|
||||
* first tries to contact the Standby NN, it could block for a long time if
|
||||
* the Standby is holding the lock for a while, e.g. when performing a
|
||||
* checkpoint. See HDFS-4591 for more details.
|
||||
*/
|
||||
void checkOperation(OperationCategory op) throws StandbyException;
|
||||
|
||||
|
|
|
@ -99,7 +99,6 @@ import org.apache.hadoop.hdfs.web.resources.UserParam;
|
|||
import org.apache.hadoop.ipc.Server;
|
||||
import org.apache.hadoop.net.NodeBase;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
import org.apache.hadoop.security.SecurityUtil;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
|
@ -212,7 +211,6 @@ public class NamenodeWebHdfsMethods {
|
|||
namenode, ugi, renewer != null? renewer: ugi.getShortUserName());
|
||||
final Token<? extends TokenIdentifier> t = c.getAllTokens().iterator().next();
|
||||
t.setKind(WebHdfsFileSystem.TOKEN_KIND);
|
||||
SecurityUtil.setTokenService(t, namenode.getHttpAddress());
|
||||
return t;
|
||||
}
|
||||
|
||||
|
|
|
@ -26,6 +26,7 @@ import java.util.Stack;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hdfs.util.XMLUtils;
|
||||
import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
|
||||
|
@ -176,7 +177,7 @@ class OfflineEditsXmlLoader
|
|||
|
||||
@Override
|
||||
public void endElement (String uri, String name, String qName) {
|
||||
String str = cbuf.toString().trim();
|
||||
String str = XMLUtils.unmangleXmlString(cbuf.toString()).trim();
|
||||
cbuf = new StringBuffer();
|
||||
switch (state) {
|
||||
case EXPECT_EDITS_TAG:
|
||||
|
|
|
@ -24,8 +24,11 @@ import java.util.Iterator;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.HadoopIllegalArgumentException;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* A low memory footprint {@link GSet} implementation,
|
||||
* which uses an array for storing the elements
|
||||
|
@ -72,7 +75,7 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
|
|||
/** Modification version for fail-fast.
|
||||
* @see ConcurrentModificationException
|
||||
*/
|
||||
private volatile int modification = 0;
|
||||
private int modification = 0;
|
||||
|
||||
/**
|
||||
* @param recommended_length Recommended size of the internal array.
|
||||
|
@ -285,4 +288,54 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
|
|||
throw new UnsupportedOperationException("Remove is not supported.");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Let t = percentage of max memory.
|
||||
* Let e = round(log_2 t).
|
||||
* Then, we choose capacity = 2^e/(size of reference),
|
||||
* unless it is outside the close interval [1, 2^30].
|
||||
*/
|
||||
public static int computeCapacity(double percentage, String mapName) {
|
||||
return computeCapacity(Runtime.getRuntime().maxMemory(), percentage,
|
||||
mapName);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
static int computeCapacity(long maxMemory, double percentage,
|
||||
String mapName) {
|
||||
if (percentage > 100.0 || percentage < 0.0) {
|
||||
throw new HadoopIllegalArgumentException("Percentage " + percentage
|
||||
+ " must be greater than or equal to 0 "
|
||||
+ " and less than or equal to 100");
|
||||
}
|
||||
if (maxMemory < 0) {
|
||||
throw new HadoopIllegalArgumentException("Memory " + maxMemory
|
||||
+ " must be greater than or equal to 0");
|
||||
}
|
||||
if (percentage == 0.0 || maxMemory == 0) {
|
||||
return 0;
|
||||
}
|
||||
//VM detection
|
||||
//See http://java.sun.com/docs/hotspot/HotSpotFAQ.html#64bit_detection
|
||||
final String vmBit = System.getProperty("sun.arch.data.model");
|
||||
|
||||
//Percentage of max memory
|
||||
final double percentDivisor = 100.0/percentage;
|
||||
final double percentMemory = maxMemory/percentDivisor;
|
||||
|
||||
//compute capacity
|
||||
final int e1 = (int)(Math.log(percentMemory)/Math.log(2.0) + 0.5);
|
||||
final int e2 = e1 - ("32".equals(vmBit)? 2: 3);
|
||||
final int exponent = e2 < 0? 0: e2 > 30? 30: e2;
|
||||
final int c = 1 << exponent;
|
||||
|
||||
if (LightWeightGSet.LOG.isDebugEnabled()) {
|
||||
LOG.debug("Computing capacity for map " + mapName);
|
||||
LOG.debug("VM type = " + vmBit + "-bit");
|
||||
LOG.debug(percentage + "% max memory = "
|
||||
+ StringUtils.TraditionalBinaryPrefix.long2String(maxMemory, "B", 1));
|
||||
LOG.debug("capacity = 2^" + exponent + " = " + c + " entries");
|
||||
}
|
||||
return c;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -46,6 +46,140 @@ public class XMLUtils {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Exception that reflects a string that cannot be unmangled.
|
||||
*/
|
||||
public static class UnmanglingError extends RuntimeException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public UnmanglingError(String str, Exception e) {
|
||||
super(str, e);
|
||||
}
|
||||
|
||||
public UnmanglingError(String str) {
|
||||
super(str);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Given a code point, determine if it should be mangled before being
|
||||
* represented in an XML document.
|
||||
*
|
||||
* Any code point that isn't valid in XML must be mangled.
|
||||
* See http://en.wikipedia.org/wiki/Valid_characters_in_XML for a
|
||||
* quick reference, or the w3 standard for the authoritative reference.
|
||||
*
|
||||
* @param cp The code point
|
||||
* @return True if the code point should be mangled
|
||||
*/
|
||||
private static boolean codePointMustBeMangled(int cp) {
|
||||
if (cp < 0x20) {
|
||||
return ((cp != 0x9) && (cp != 0xa) && (cp != 0xd));
|
||||
} else if ((0xd7ff < cp) && (cp < 0xe000)) {
|
||||
return true;
|
||||
} else if ((cp == 0xfffe) || (cp == 0xffff)) {
|
||||
return true;
|
||||
} else if (cp == 0x5c) {
|
||||
// we mangle backslash to simplify decoding... it's
|
||||
// easier if backslashes always begin mangled sequences.
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private static int NUM_SLASH_POSITIONS = 4;
|
||||
|
||||
private static String mangleCodePoint(int cp) {
|
||||
return String.format("\\%0" + NUM_SLASH_POSITIONS + "x;", cp);
|
||||
}
|
||||
|
||||
/**
|
||||
* Mangle a string so that it can be represented in an XML document.
|
||||
*
|
||||
* There are three kinds of code points in XML:
|
||||
* - Those that can be represented normally,
|
||||
* - Those that have to be escaped (for example, & must be represented
|
||||
* as &)
|
||||
* - Those that cannot be represented at all in XML.
|
||||
*
|
||||
* The built-in SAX functions will handle the first two types for us just
|
||||
* fine. However, sometimes we come across a code point of the third type.
|
||||
* In this case, we have to mangle the string in order to represent it at
|
||||
* all. We also mangle backslash to avoid confusing a backslash in the
|
||||
* string with part our escape sequence.
|
||||
*
|
||||
* The encoding used here is as follows: an illegal code point is
|
||||
* represented as '\ABCD;', where ABCD is the hexadecimal value of
|
||||
* the code point.
|
||||
*
|
||||
* @param str The input string.
|
||||
*
|
||||
* @return The mangled string.
|
||||
*/
|
||||
public static String mangleXmlString(String str) {
|
||||
final StringBuilder bld = new StringBuilder();
|
||||
final int length = str.length();
|
||||
for (int offset = 0; offset < length; ) {
|
||||
final int cp = str.codePointAt(offset);
|
||||
final int len = Character.charCount(cp);
|
||||
if (codePointMustBeMangled(cp)) {
|
||||
bld.append(mangleCodePoint(cp));
|
||||
} else {
|
||||
for (int i = 0; i < len; i++) {
|
||||
bld.append(str.charAt(offset + i));
|
||||
}
|
||||
}
|
||||
offset += len;
|
||||
}
|
||||
return bld.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Demangle a string from an XML document.
|
||||
* See {@link #mangleXmlString(String)} for a description of the mangling
|
||||
* format.
|
||||
*
|
||||
* @param str The string to be demangled.
|
||||
*
|
||||
* @return The unmangled string
|
||||
* @throws UnmanglingError if the input is malformed.
|
||||
*/
|
||||
public static String unmangleXmlString(String str)
|
||||
throws UnmanglingError {
|
||||
int slashPosition = -1;
|
||||
String escapedCp = "";
|
||||
StringBuilder bld = new StringBuilder();
|
||||
for (int i = 0; i < str.length(); i++) {
|
||||
char ch = str.charAt(i);
|
||||
if ((slashPosition >= 0) && (slashPosition < NUM_SLASH_POSITIONS)) {
|
||||
escapedCp += ch;
|
||||
++slashPosition;
|
||||
} else if (slashPosition == NUM_SLASH_POSITIONS) {
|
||||
if (ch != ';') {
|
||||
throw new UnmanglingError("unterminated code point escape: " +
|
||||
"expected semicolon at end.");
|
||||
}
|
||||
try {
|
||||
bld.appendCodePoint(Integer.parseInt(escapedCp, 16));
|
||||
} catch (NumberFormatException e) {
|
||||
throw new UnmanglingError("error parsing unmangling escape code", e);
|
||||
}
|
||||
escapedCp = "";
|
||||
slashPosition = -1;
|
||||
} else if (ch == '\\') {
|
||||
slashPosition = 0;
|
||||
} else {
|
||||
bld.append(ch);
|
||||
}
|
||||
}
|
||||
if (slashPosition != -1) {
|
||||
throw new UnmanglingError("unterminated code point escape: string " +
|
||||
"broke off in the middle");
|
||||
}
|
||||
return bld.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a SAX tag with a string inside.
|
||||
*
|
||||
|
@ -56,7 +190,7 @@ public class XMLUtils {
|
|||
public static void addSaxString(ContentHandler contentHandler,
|
||||
String tag, String val) throws SAXException {
|
||||
contentHandler.startElement("", "", tag, new AttributesImpl());
|
||||
char c[] = val.toString().toCharArray();
|
||||
char c[] = mangleXmlString(val).toCharArray();
|
||||
contentHandler.characters(c, 0, c.length);
|
||||
contentHandler.endElement("", "", tag);
|
||||
}
|
||||
|
@ -67,6 +201,8 @@ public class XMLUtils {
|
|||
*/
|
||||
static public class Stanza {
|
||||
private TreeMap<String, LinkedList <Stanza > > subtrees;
|
||||
|
||||
/** The unmangled value of this stanza. */
|
||||
private String value;
|
||||
|
||||
public Stanza() {
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.server.namenode.INodeId;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
|
@ -244,7 +245,8 @@ public class JsonUtil {
|
|||
final long mTime = (Long) m.get("modificationTime");
|
||||
final long blockSize = (Long) m.get("blockSize");
|
||||
final short replication = (short) (long) (Long) m.get("replication");
|
||||
final long fileId = (Long) m.get("fileId");
|
||||
final long fileId = m.containsKey("fileId") ? (Long) m.get("fileId")
|
||||
: INodeId.GRANDFATHER_INODE_ID;
|
||||
return new HdfsFileStatus(len, type == PathType.DIRECTORY, replication,
|
||||
blockSize, mTime, aTime, permission, owner, group,
|
||||
symlink, DFSUtil.string2Bytes(localName), fileId);
|
||||
|
|
|
@ -29,7 +29,7 @@ import java.net.MalformedURLException;
|
|||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.net.URL;
|
||||
import java.util.ArrayList;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -62,15 +62,16 @@ import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
|
|||
import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
|
||||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
|
||||
import org.apache.hadoop.hdfs.server.common.JspHelper;
|
||||
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
|
||||
import org.apache.hadoop.hdfs.web.resources.AccessTimeParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.ConcatSourcesParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.CreateParentParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.DelegationParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.DeleteOpParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.DestinationParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.DoAsParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.GetOpParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.GroupParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
|
||||
|
@ -106,10 +107,11 @@ import org.apache.hadoop.security.token.TokenIdentifier;
|
|||
import org.apache.hadoop.security.token.TokenRenewer;
|
||||
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
|
||||
import org.apache.hadoop.util.Progressable;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.mortbay.util.ajax.JSON;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/** A FileSystem for HDFS over the web. */
|
||||
public class WebHdfsFileSystem extends FileSystem
|
||||
|
@ -132,7 +134,8 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
|
||||
private DelegationTokenRenewer dtRenewer = null;
|
||||
|
||||
private synchronized void addRenewAction(final WebHdfsFileSystem webhdfs) {
|
||||
@VisibleForTesting
|
||||
protected synchronized void addRenewAction(final WebHdfsFileSystem webhdfs) {
|
||||
if (dtRenewer == null) {
|
||||
dtRenewer = DelegationTokenRenewer.getInstance();
|
||||
}
|
||||
|
@ -148,22 +151,15 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
return b;
|
||||
}
|
||||
|
||||
private final UserGroupInformation ugi;
|
||||
private UserGroupInformation ugi;
|
||||
private InetSocketAddress nnAddr;
|
||||
private URI uri;
|
||||
private boolean hasInitedToken;
|
||||
private Token<?> delegationToken;
|
||||
private final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
|
||||
private RetryPolicy retryPolicy = null;
|
||||
private Path workingDir;
|
||||
|
||||
{
|
||||
try {
|
||||
ugi = UserGroupInformation.getCurrentUser();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the protocol scheme for the FileSystem.
|
||||
* <p/>
|
||||
|
@ -180,6 +176,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
) throws IOException {
|
||||
super.initialize(uri, conf);
|
||||
setConf(conf);
|
||||
ugi = UserGroupInformation.getCurrentUser();
|
||||
try {
|
||||
this.uri = new URI(uri.getScheme(), uri.getAuthority(), null, null, null);
|
||||
} catch (URISyntaxException e) {
|
||||
|
@ -204,24 +201,26 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
protected void initDelegationToken() throws IOException {
|
||||
// look for webhdfs token, then try hdfs
|
||||
Token<?> token = selectDelegationToken(ugi);
|
||||
|
||||
//since we don't already have a token, go get one
|
||||
boolean createdToken = false;
|
||||
if (token == null) {
|
||||
token = getDelegationToken(null);
|
||||
createdToken = (token != null);
|
||||
}
|
||||
|
||||
// security might be disabled
|
||||
if (token != null) {
|
||||
LOG.debug("Found existing DT for " + token.getService());
|
||||
setDelegationToken(token);
|
||||
if (createdToken) {
|
||||
hasInitedToken = true;
|
||||
}
|
||||
}
|
||||
|
||||
protected synchronized Token<?> getDelegationToken() throws IOException {
|
||||
if (!hasInitedToken) {
|
||||
//since we don't already have a token, go get one
|
||||
Token<?> token = getDelegationToken(null);
|
||||
// security might be disabled
|
||||
if (token != null) {
|
||||
setDelegationToken(token);
|
||||
addRenewAction(this);
|
||||
LOG.debug("Created new DT for " + token.getService());
|
||||
} else {
|
||||
LOG.debug("Found existing DT for " + token.getService());
|
||||
}
|
||||
hasInitedToken = true;
|
||||
}
|
||||
return delegationToken;
|
||||
}
|
||||
|
||||
protected Token<DelegationTokenIdentifier> selectDelegationToken(
|
||||
|
@ -239,6 +238,11 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
public URI getUri() {
|
||||
return this.uri;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected URI canonicalizeUri(URI uri) {
|
||||
return NetUtils.getCanonicalUri(uri, getDefaultPort());
|
||||
}
|
||||
|
||||
/** @return the home directory. */
|
||||
public static String getHomeDirectoryString(final UserGroupInformation ugi) {
|
||||
|
@ -365,16 +369,26 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
return url;
|
||||
}
|
||||
|
||||
private String addDt2Query(String query) throws IOException {
|
||||
if (UserGroupInformation.isSecurityEnabled()) {
|
||||
synchronized (this) {
|
||||
if (delegationToken != null) {
|
||||
final String encoded = delegationToken.encodeToUrlString();
|
||||
return query + JspHelper.getDelegationTokenUrlParam(encoded);
|
||||
} // else we are talking to an insecure cluster
|
||||
}
|
||||
Param<?,?>[] getAuthParameters(final HttpOpParam.Op op) throws IOException {
|
||||
List<Param<?,?>> authParams = Lists.newArrayList();
|
||||
// Skip adding delegation token for token operations because these
|
||||
// operations require authentication.
|
||||
Token<?> token = null;
|
||||
if (UserGroupInformation.isSecurityEnabled() && !op.getRequireAuth()) {
|
||||
token = getDelegationToken();
|
||||
}
|
||||
return query;
|
||||
if (token != null) {
|
||||
authParams.add(new DelegationParam(token.encodeToUrlString()));
|
||||
} else {
|
||||
UserGroupInformation userUgi = ugi;
|
||||
UserGroupInformation realUgi = userUgi.getRealUser();
|
||||
if (realUgi != null) { // proxy user
|
||||
authParams.add(new DoAsParam(userUgi.getShortUserName()));
|
||||
userUgi = realUgi;
|
||||
}
|
||||
authParams.add(new UserParam(userUgi.getShortUserName()));
|
||||
}
|
||||
return authParams.toArray(new Param<?,?>[0]);
|
||||
}
|
||||
|
||||
URL toUrl(final HttpOpParam.Op op, final Path fspath,
|
||||
|
@ -383,34 +397,15 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
final String path = PATH_PREFIX
|
||||
+ (fspath == null? "/": makeQualified(fspath).toUri().getPath());
|
||||
final String query = op.toQueryString()
|
||||
+ '&' + new UserParam(ugi)
|
||||
+ Param.toSortedString("&", getAuthParameters(op))
|
||||
+ Param.toSortedString("&", parameters);
|
||||
final URL url;
|
||||
if (op == PutOpParam.Op.RENEWDELEGATIONTOKEN
|
||||
|| op == GetOpParam.Op.GETDELEGATIONTOKEN) {
|
||||
// Skip adding delegation token for getting or renewing delegation token,
|
||||
// because these operations require kerberos authentication.
|
||||
url = getNamenodeURL(path, query);
|
||||
} else {
|
||||
url = getNamenodeURL(path, addDt2Query(query));
|
||||
}
|
||||
final URL url = getNamenodeURL(path, query);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("url=" + url);
|
||||
}
|
||||
return url;
|
||||
}
|
||||
|
||||
private HttpURLConnection getHttpUrlConnection(URL url)
|
||||
throws IOException, AuthenticationException {
|
||||
final HttpURLConnection conn;
|
||||
if (ugi.hasKerberosCredentials()) {
|
||||
conn = new AuthenticatedURL(AUTH).openConnection(url, authToken);
|
||||
} else {
|
||||
conn = (HttpURLConnection)url.openConnection();
|
||||
}
|
||||
return conn;
|
||||
}
|
||||
|
||||
/**
|
||||
* Run a http operation.
|
||||
* Connect to the http server, validate response, and obtain the JSON output.
|
||||
|
@ -455,6 +450,48 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
this.conn = conn;
|
||||
}
|
||||
|
||||
private HttpURLConnection getHttpUrlConnection(final URL url)
|
||||
throws IOException, AuthenticationException {
|
||||
UserGroupInformation connectUgi = ugi.getRealUser();
|
||||
if (connectUgi == null) {
|
||||
connectUgi = ugi;
|
||||
}
|
||||
try {
|
||||
return connectUgi.doAs(
|
||||
new PrivilegedExceptionAction<HttpURLConnection>() {
|
||||
@Override
|
||||
public HttpURLConnection run() throws IOException {
|
||||
return openHttpUrlConnection(url);
|
||||
}
|
||||
});
|
||||
} catch (IOException ioe) {
|
||||
Throwable cause = ioe.getCause();
|
||||
if (cause != null && cause instanceof AuthenticationException) {
|
||||
throw (AuthenticationException)cause;
|
||||
}
|
||||
throw ioe;
|
||||
} catch (InterruptedException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private HttpURLConnection openHttpUrlConnection(final URL url)
|
||||
throws IOException {
|
||||
final HttpURLConnection conn;
|
||||
try {
|
||||
if (op.getRequireAuth()) {
|
||||
LOG.debug("open AuthenticatedURL connection");
|
||||
conn = new AuthenticatedURL(AUTH).openConnection(url, authToken);
|
||||
} else {
|
||||
LOG.debug("open URL connection");
|
||||
conn = (HttpURLConnection)url.openConnection();
|
||||
}
|
||||
} catch (AuthenticationException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
return conn;
|
||||
}
|
||||
|
||||
private void init() throws IOException {
|
||||
checkRetry = !redirected;
|
||||
try {
|
||||
|
@ -721,17 +758,10 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
}
|
||||
|
||||
@Override
|
||||
public void concat(final Path trg, final Path [] psrcs) throws IOException {
|
||||
public void concat(final Path trg, final Path [] srcs) throws IOException {
|
||||
statistics.incrementWriteOps(1);
|
||||
final HttpOpParam.Op op = PostOpParam.Op.CONCAT;
|
||||
|
||||
List<String> strPaths = new ArrayList<String>(psrcs.length);
|
||||
for(Path psrc : psrcs) {
|
||||
strPaths.add(psrc.toUri().getPath());
|
||||
}
|
||||
|
||||
String srcs = StringUtils.join(",", strPaths);
|
||||
|
||||
ConcatSourcesParam param = new ConcatSourcesParam(srcs);
|
||||
run(op, trg, param);
|
||||
}
|
||||
|
|
|
@ -18,15 +18,28 @@
|
|||
|
||||
package org.apache.hadoop.hdfs.web.resources;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
/** The concat source paths parameter. */
|
||||
public class ConcatSourcesParam extends StringParam {
|
||||
/** Parameter name. */
|
||||
public static final String NAME = "sources";
|
||||
|
||||
public static final String DEFAULT = NULL;
|
||||
public static final String DEFAULT = "";
|
||||
|
||||
private static final Domain DOMAIN = new Domain(NAME, null);
|
||||
|
||||
private static String paths2String(Path[] paths) {
|
||||
if (paths == null || paths.length == 0) {
|
||||
return "";
|
||||
}
|
||||
final StringBuilder b = new StringBuilder(paths[0].toUri().getPath());
|
||||
for(int i = 1; i < paths.length; i++) {
|
||||
b.append(',').append(paths[i].toUri().getPath());
|
||||
}
|
||||
return b.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor.
|
||||
* @param str a string representation of the parameter value.
|
||||
|
@ -35,6 +48,10 @@ public class ConcatSourcesParam extends StringParam {
|
|||
super(DOMAIN, str);
|
||||
}
|
||||
|
||||
public ConcatSourcesParam(Path[] paths) {
|
||||
this(paths2String(paths));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return NAME;
|
||||
|
|
|
@ -38,6 +38,11 @@ public class DeleteOpParam extends HttpOpParam<DeleteOpParam.Op> {
|
|||
return HttpOpParam.Type.DELETE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getRequireAuth() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getDoOutput() {
|
||||
return false;
|
||||
|
|
|
@ -31,7 +31,7 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
|
|||
GETFILECHECKSUM(true, HttpURLConnection.HTTP_OK),
|
||||
|
||||
GETHOMEDIRECTORY(false, HttpURLConnection.HTTP_OK),
|
||||
GETDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK),
|
||||
GETDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK, true),
|
||||
|
||||
/** GET_BLOCK_LOCATIONS is a private unstable op. */
|
||||
GET_BLOCK_LOCATIONS(false, HttpURLConnection.HTTP_OK),
|
||||
|
@ -40,16 +40,28 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
|
|||
|
||||
final boolean redirect;
|
||||
final int expectedHttpResponseCode;
|
||||
final boolean requireAuth;
|
||||
|
||||
Op(final boolean redirect, final int expectedHttpResponseCode) {
|
||||
this(redirect, expectedHttpResponseCode, false);
|
||||
}
|
||||
|
||||
Op(final boolean redirect, final int expectedHttpResponseCode,
|
||||
final boolean requireAuth) {
|
||||
this.redirect = redirect;
|
||||
this.expectedHttpResponseCode = expectedHttpResponseCode;
|
||||
this.requireAuth = requireAuth;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HttpOpParam.Type getType() {
|
||||
return HttpOpParam.Type.GET;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getRequireAuth() {
|
||||
return requireAuth;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getDoOutput() {
|
||||
|
|
|
@ -43,6 +43,9 @@ public abstract class HttpOpParam<E extends Enum<E> & HttpOpParam.Op>
|
|||
/** @return the Http operation type. */
|
||||
public Type getType();
|
||||
|
||||
/** @return true if the operation cannot use a token */
|
||||
public boolean getRequireAuth();
|
||||
|
||||
/** @return true if the operation will do output. */
|
||||
public boolean getDoOutput();
|
||||
|
||||
|
@ -92,6 +95,11 @@ public abstract class HttpOpParam<E extends Enum<E> & HttpOpParam.Op>
|
|||
return op.getType();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getRequireAuth() {
|
||||
return op.getRequireAuth();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getDoOutput() {
|
||||
return op.getDoOutput();
|
||||
|
|
|
@ -41,6 +41,11 @@ public class PostOpParam extends HttpOpParam<PostOpParam.Op> {
|
|||
public Type getType() {
|
||||
return Type.POST;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getRequireAuth() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getDoOutput() {
|
||||
|
|
|
@ -34,23 +34,35 @@ public class PutOpParam extends HttpOpParam<PutOpParam.Op> {
|
|||
SETPERMISSION(false, HttpURLConnection.HTTP_OK),
|
||||
SETTIMES(false, HttpURLConnection.HTTP_OK),
|
||||
|
||||
RENEWDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK),
|
||||
CANCELDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK),
|
||||
RENEWDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK, true),
|
||||
CANCELDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK, true),
|
||||
|
||||
NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED);
|
||||
|
||||
final boolean doOutputAndRedirect;
|
||||
final int expectedHttpResponseCode;
|
||||
final boolean requireAuth;
|
||||
|
||||
Op(final boolean doOutputAndRedirect, final int expectedHttpResponseCode) {
|
||||
this(doOutputAndRedirect, expectedHttpResponseCode, false);
|
||||
}
|
||||
|
||||
Op(final boolean doOutputAndRedirect, final int expectedHttpResponseCode,
|
||||
final boolean requireAuth) {
|
||||
this.doOutputAndRedirect = doOutputAndRedirect;
|
||||
this.expectedHttpResponseCode = expectedHttpResponseCode;
|
||||
this.requireAuth = requireAuth;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HttpOpParam.Type getType() {
|
||||
return HttpOpParam.Type.PUT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getRequireAuth() {
|
||||
return requireAuth;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getDoOutput() {
|
||||
|
|
|
@ -60,9 +60,7 @@ that process vast amounts of data. Here's what makes Hadoop especially useful:</
|
|||
Hadoop was been demonstrated on GNU/Linux clusters with 2000 nodes.
|
||||
</li>
|
||||
<li>
|
||||
Win32 is supported as a <i>development</i> platform. Distributed operation
|
||||
has not been well tested on Win32, so this is not a <i>production</i>
|
||||
platform.
|
||||
Windows is also a supported platform.
|
||||
</li>
|
||||
</ul>
|
||||
|
||||
|
@ -84,15 +82,6 @@ that process vast amounts of data. Here's what makes Hadoop especially useful:</
|
|||
</li>
|
||||
</ol>
|
||||
|
||||
<h4>Additional requirements for Windows</h4>
|
||||
|
||||
<ol>
|
||||
<li>
|
||||
<a href="http://www.cygwin.com/">Cygwin</a> - Required for shell support in
|
||||
addition to the required software above.
|
||||
</li>
|
||||
</ol>
|
||||
|
||||
<h3>Installing Required Software</h3>
|
||||
|
||||
<p>If your platform does not have the required software listed above, you
|
||||
|
@ -104,13 +93,6 @@ $ sudo apt-get install ssh<br>
|
|||
$ sudo apt-get install rsync<br>
|
||||
</pre></blockquote></p>
|
||||
|
||||
<p>On Windows, if you did not install the required software when you
|
||||
installed cygwin, start the cygwin installer and select the packages:</p>
|
||||
<ul>
|
||||
<li>openssh - the "Net" category</li>
|
||||
<li>rsync - the "Net" category</li>
|
||||
</ul>
|
||||
|
||||
<h2>Getting Started</h2>
|
||||
|
||||
<p>First, you need to get a copy of the Hadoop code.</p>
|
||||
|
|
|
@ -82,7 +82,7 @@ unset IFS
|
|||
findlibjvm () {
|
||||
javabasedir=$JAVA_HOME
|
||||
case $OS_NAME in
|
||||
cygwin* | mingw* | pw23* )
|
||||
mingw* | pw23* )
|
||||
lib_jvm_dir=`find $javabasedir -follow \( \
|
||||
\( -name client -type d -prune \) -o \
|
||||
\( -name "jvm.dll" -exec dirname {} \; \) \) 2> /dev/null | tr "\n" " "`
|
||||
|
|
|
@ -47,7 +47,7 @@
|
|||
RPC address that handles all clients requests. In the case of HA/Federation where multiple namenodes exist,
|
||||
the name service id is added to the name e.g. dfs.namenode.rpc-address.ns1
|
||||
dfs.namenode.rpc-address.EXAMPLENAMESERVICE
|
||||
The value of this property will take the form of hdfs://nn-host1:rpc-port.
|
||||
The value of this property will take the form of nn-host1:rpc-port.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
|
@ -59,7 +59,7 @@
|
|||
connecting to this address if it is configured. In the case of HA/Federation where multiple namenodes exist,
|
||||
the name service id is added to the name e.g. dfs.namenode.servicerpc-address.ns1
|
||||
dfs.namenode.rpc-address.EXAMPLENAMESERVICE
|
||||
The value of this property will take the form of hdfs://nn-host1:rpc-port.
|
||||
The value of this property will take the form of nn-host1:rpc-port.
|
||||
If the value of this property is unset the value of dfs.namenode.rpc-address will be used as the default.
|
||||
</description>
|
||||
</property>
|
||||
|
@ -594,6 +594,17 @@
|
|||
<description>Packet size for clients to write</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.client.write.exclude.nodes.cache.expiry.interval.millis</name>
|
||||
<value>600000</value>
|
||||
<description>The maximum period to keep a DN in the excluded nodes list
|
||||
at a client. After this period, in milliseconds, the previously excluded node(s) will
|
||||
be removed automatically from the cache and will be considered good for block allocations
|
||||
again. Useful to lower or raise in situations where you keep a file open for very long
|
||||
periods (such as a Write-Ahead-Log (WAL) file) to make the writer tolerant to cluster maintenance
|
||||
restarts. Defaults to 10 minutes.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.namenode.checkpoint.dir</name>
|
||||
<value>file://${hadoop.tmp.dir}/dfs/namesecondary</value>
|
||||
|
@ -624,7 +635,7 @@
|
|||
|
||||
<property>
|
||||
<name>dfs.namenode.checkpoint.txns</name>
|
||||
<value>40000</value>
|
||||
<value>1000000</value>
|
||||
<description>The Secondary NameNode or CheckpointNode will create a checkpoint
|
||||
of the namespace every 'dfs.namenode.checkpoint.txns' transactions, regardless
|
||||
of whether 'dfs.namenode.checkpoint.period' has expired.
|
||||
|
@ -640,6 +651,15 @@
|
|||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.namenode.checkpoint.max-retries</name>
|
||||
<value>3</value>
|
||||
<description>The SecondaryNameNode retries failed checkpointing. If the
|
||||
failure occurs while loading fsimage or replaying edits, the number of
|
||||
retries is limited by this variable.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.namenode.num.checkpoints.retained</name>
|
||||
<value>2</value>
|
||||
|
@ -724,12 +744,29 @@
|
|||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.image.transfer.timeout</name>
|
||||
<value>600000</value>
|
||||
<description>
|
||||
Timeout for image transfer in milliseconds. This timeout and the related
|
||||
dfs.image.transfer.bandwidthPerSec parameter should be configured such
|
||||
that normal image transfer can complete within the timeout.
|
||||
This timeout prevents client hangs when the sender fails during
|
||||
image transfer, which is particularly important during checkpointing.
|
||||
Note that this timeout applies to the entirety of image transfer, and
|
||||
is not a socket timeout.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.image.transfer.bandwidthPerSec</name>
|
||||
<value>0</value>
|
||||
<description>
|
||||
Specifies the maximum amount of bandwidth that can be utilized for image
|
||||
transfer in term of the number of bytes per second.
|
||||
Maximum bandwidth used for image transfer in bytes per second.
|
||||
This can help keep normal namenode operations responsive during
|
||||
checkpointing. The maximum bandwidth and timeout in
|
||||
dfs.image.transfer.timeout should be set such that normal image
|
||||
transfers can complete successfully.
|
||||
A default value of 0 indicates that throttling is disabled.
|
||||
</description>
|
||||
</property>
|
||||
|
|
|
@ -193,7 +193,7 @@ HDFS Users Guide
|
|||
* <<<dfs.namenode.checkpoint.period>>>, set to 1 hour by default, specifies
|
||||
the maximum delay between two consecutive checkpoints, and
|
||||
|
||||
* <<<dfs.namenode.checkpoint.txns>>>, set to 40000 default, defines the
|
||||
* <<<dfs.namenode.checkpoint.txns>>>, set to 1 million by default, defines the
|
||||
number of uncheckpointed transactions on the NameNode which will
|
||||
force an urgent checkpoint, even if the checkpoint period has not
|
||||
been reached.
|
||||
|
@ -232,7 +232,7 @@ HDFS Users Guide
|
|||
* <<<dfs.namenode.checkpoint.period>>>, set to 1 hour by default, specifies
|
||||
the maximum delay between two consecutive checkpoints
|
||||
|
||||
* <<<dfs.namenode.checkpoint.txns>>>, set to 40000 default, defines the
|
||||
* <<<dfs.namenode.checkpoint.txns>>>, set to 1 million by default, defines the
|
||||
number of uncheckpointed transactions on the NameNode which will
|
||||
force an urgent checkpoint, even if the checkpoint period has not
|
||||
been reached.
|
||||
|
|
|
@ -109,7 +109,7 @@ WebHDFS REST API
|
|||
* {{{Append to a File}<<<APPEND>>>}}
|
||||
(see {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.append)
|
||||
|
||||
* {{{Concat File(s)}<<<CONCAT>>>}}
|
||||
* {{{Concatenate Files}<<<CONCAT>>>}}
|
||||
(see {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.concat)
|
||||
|
||||
* HTTP DELETE
|
||||
|
@ -307,7 +307,7 @@ Content-Length: 0
|
|||
* Submit a HTTP POST request.
|
||||
|
||||
+---------------------------------
|
||||
curl -i -X POST "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=CONCAT&sources=<SOURCES>"
|
||||
curl -i -X POST "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=CONCAT&sources=<PATHS>"
|
||||
+---------------------------------
|
||||
|
||||
The client receives a response with zero content length:
|
||||
|
@ -319,10 +319,6 @@ Content-Length: 0
|
|||
|
||||
[]
|
||||
|
||||
This REST API call is available as of Hadoop version 2.0.3.
|
||||
Please note that <SOURCES> is a comma seperated list of absolute paths.
|
||||
(Example: sources=/test/file1,/test/file2,/test/file3)
|
||||
|
||||
See also:
|
||||
{{{Sources}<<<sources>>>}},
|
||||
{{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.concat
|
||||
|
@ -1761,7 +1757,7 @@ var tokenProperties =
|
|||
*----------------+-------------------------------------------------------------------+
|
||||
|| Name | <<<sources>>> |
|
||||
*----------------+-------------------------------------------------------------------+
|
||||
|| Description | The comma seperated absolute paths used for concatenation. |
|
||||
|| Description | A list of source paths. |
|
||||
*----------------+-------------------------------------------------------------------+
|
||||
|| Type | String |
|
||||
*----------------+-------------------------------------------------------------------+
|
||||
|
@ -1769,12 +1765,9 @@ var tokenProperties =
|
|||
*----------------+-------------------------------------------------------------------+
|
||||
|| Valid Values | A list of comma seperated absolute FileSystem paths without scheme and authority. |
|
||||
*----------------+-------------------------------------------------------------------+
|
||||
|| Syntax | See the note in {{Delegation}}. |
|
||||
|| Syntax | Any string. |
|
||||
*----------------+-------------------------------------------------------------------+
|
||||
|
||||
<<Note>> that sources are absolute FileSystem paths.
|
||||
|
||||
|
||||
See also:
|
||||
{{{Concat File(s)}<<<CONCAT>>>}}
|
||||
|
||||
|
|
|
@ -0,0 +1,64 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.log4j.AppenderSkeleton;
|
||||
import org.apache.log4j.spi.LoggingEvent;
|
||||
import org.apache.log4j.spi.ThrowableInformation;
|
||||
|
||||
/**
|
||||
* Used to verify that certain exceptions or messages are present in log output.
|
||||
*/
|
||||
public class LogVerificationAppender extends AppenderSkeleton {
|
||||
private final List<LoggingEvent> log = new ArrayList<LoggingEvent>();
|
||||
|
||||
@Override
|
||||
public boolean requiresLayout() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void append(final LoggingEvent loggingEvent) {
|
||||
log.add(loggingEvent);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
}
|
||||
|
||||
public List<LoggingEvent> getLog() {
|
||||
return new ArrayList<LoggingEvent>(log);
|
||||
}
|
||||
|
||||
public int countExceptionsWithMessage(final String text) {
|
||||
int count = 0;
|
||||
for (LoggingEvent e: getLog()) {
|
||||
ThrowableInformation t = e.getThrowableInformation();
|
||||
if (t != null) {
|
||||
String m = t.getThrowable().getMessage();
|
||||
if (m.contains(text)) {
|
||||
count++;
|
||||
}
|
||||
}
|
||||
}
|
||||
return count;
|
||||
}
|
||||
}
|
|
@ -21,36 +21,134 @@ import static org.junit.Assert.fail;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.util.List;
|
||||
|
||||
import junit.framework.Assert;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
|
||||
import org.apache.hadoop.util.ThreadUtil;
|
||||
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
|
||||
/**
|
||||
* These tests make sure that DFSClient retries fetching data from DFS
|
||||
* properly in case of errors.
|
||||
* These tests make sure that DFSClient excludes writing data to
|
||||
* a DN properly in case of errors.
|
||||
*/
|
||||
public class TestDFSClientExcludedNodes {
|
||||
|
||||
@Test
|
||||
private MiniDFSCluster cluster;
|
||||
private Configuration conf;
|
||||
|
||||
@Before
|
||||
public void setUp() {
|
||||
cluster = null;
|
||||
conf = new HdfsConfiguration();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testExcludedNodes() throws IOException {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
Path filePath = new Path("/testExcludedNodes");
|
||||
|
||||
// kill a datanode
|
||||
cluster.stopDataNode(AppendTestUtil.nextInt(3));
|
||||
OutputStream out = fs.create(filePath, true, 4096);
|
||||
OutputStream out = fs.create(
|
||||
filePath,
|
||||
true,
|
||||
4096,
|
||||
(short) 3,
|
||||
fs.getDefaultBlockSize(filePath)
|
||||
);
|
||||
out.write(20);
|
||||
|
||||
try {
|
||||
out.close();
|
||||
} catch (Exception e) {
|
||||
fail("DataNode failure should not result in a block abort: \n" + e.getMessage());
|
||||
fail("Single DN failure should not result in a block abort: \n" +
|
||||
e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testExcludedNodesForgiveness() throws IOException {
|
||||
// Forgive nodes in under 2.5s for this test case.
|
||||
conf.setLong(
|
||||
DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL,
|
||||
2500);
|
||||
// We'll be using a 512 bytes block size just for tests
|
||||
// so making sure the checksum bytes too match it.
|
||||
conf.setInt("io.bytes.per.checksum", 512);
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
|
||||
List<DataNodeProperties> props = cluster.dataNodes;
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
Path filePath = new Path("/testForgivingExcludedNodes");
|
||||
|
||||
// 256 bytes data chunk for writes
|
||||
byte[] bytes = new byte[256];
|
||||
for (int index=0; index<bytes.length; index++) {
|
||||
bytes[index] = '0';
|
||||
}
|
||||
|
||||
// File with a 512 bytes block size
|
||||
FSDataOutputStream out = fs.create(filePath, true, 4096, (short) 3, 512);
|
||||
|
||||
// Write a block to all 3 DNs (2x256bytes).
|
||||
out.write(bytes);
|
||||
out.write(bytes);
|
||||
out.hflush();
|
||||
|
||||
// Remove two DNs, to put them into the exclude list.
|
||||
DataNodeProperties two = cluster.stopDataNode(2);
|
||||
DataNodeProperties one = cluster.stopDataNode(1);
|
||||
|
||||
// Write another block.
|
||||
// At this point, we have two nodes already in excluded list.
|
||||
out.write(bytes);
|
||||
out.write(bytes);
|
||||
out.hflush();
|
||||
|
||||
// Bring back the older DNs, since they are gonna be forgiven only
|
||||
// afterwards of this previous block write.
|
||||
Assert.assertEquals(true, cluster.restartDataNode(one, true));
|
||||
Assert.assertEquals(true, cluster.restartDataNode(two, true));
|
||||
cluster.waitActive();
|
||||
|
||||
// Sleep for 5s, to let the excluded nodes be expired
|
||||
// from the excludes list (i.e. forgiven after the configured wait period).
|
||||
// [Sleeping just in case the restart of the DNs completed < 5s cause
|
||||
// otherwise, we'll end up quickly excluding those again.]
|
||||
ThreadUtil.sleepAtLeastIgnoreInterrupts(5000);
|
||||
|
||||
// Terminate the last good DN, to assert that there's no
|
||||
// single-DN-available scenario, caused by not forgiving the other
|
||||
// two by now.
|
||||
cluster.stopDataNode(0);
|
||||
|
||||
try {
|
||||
// Attempt writing another block, which should still pass
|
||||
// cause the previous two should have been forgiven by now,
|
||||
// while the last good DN added to excludes this time.
|
||||
out.write(bytes);
|
||||
out.hflush();
|
||||
out.close();
|
||||
} catch (Exception e) {
|
||||
fail("Excluded DataNodes should be forgiven after a while and " +
|
||||
"not cause file writing exception of: '" + e.getMessage() + "'");
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -26,8 +26,11 @@ import java.io.IOException;
|
|||
import java.net.InetSocketAddress;
|
||||
import java.net.Socket;
|
||||
import java.net.SocketAddress;
|
||||
import java.lang.reflect.Field;
|
||||
import java.net.InetAddress;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.List;
|
||||
|
||||
import javax.net.SocketFactory;
|
||||
|
||||
|
@ -35,6 +38,7 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
||||
import org.apache.hadoop.fs.FileContext;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||
|
@ -48,10 +52,13 @@ import org.apache.hadoop.util.StringUtils;
|
|||
import org.hamcrest.BaseMatcher;
|
||||
import org.hamcrest.Description;
|
||||
import org.junit.After;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import sun.net.spi.nameservice.NameService;
|
||||
|
||||
public class TestDFSClientFailover {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(TestDFSClientFailover.class);
|
||||
|
@ -201,4 +208,74 @@ public class TestDFSClientFailover {
|
|||
"Could not find any configured addresses for URI " + uri));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Spy on the Java DNS infrastructure.
|
||||
* This likely only works on Sun-derived JDKs, but uses JUnit's
|
||||
* Assume functionality so that any tests using it are skipped on
|
||||
* incompatible JDKs.
|
||||
*/
|
||||
private NameService spyOnNameService() {
|
||||
try {
|
||||
Field f = InetAddress.class.getDeclaredField("nameServices");
|
||||
f.setAccessible(true);
|
||||
Assume.assumeNotNull(f);
|
||||
@SuppressWarnings("unchecked")
|
||||
List<NameService> nsList = (List<NameService>) f.get(null);
|
||||
|
||||
NameService ns = nsList.get(0);
|
||||
Log log = LogFactory.getLog("NameServiceSpy");
|
||||
|
||||
ns = Mockito.mock(NameService.class,
|
||||
new GenericTestUtils.DelegateAnswer(log, ns));
|
||||
nsList.set(0, ns);
|
||||
return ns;
|
||||
} catch (Throwable t) {
|
||||
LOG.info("Unable to spy on DNS. Skipping test.", t);
|
||||
// In case the JDK we're testing on doesn't work like Sun's, just
|
||||
// skip the test.
|
||||
Assume.assumeNoException(t);
|
||||
throw new RuntimeException(t);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that the client doesn't ever try to DNS-resolve the logical URI.
|
||||
* Regression test for HADOOP-9150.
|
||||
*/
|
||||
@Test
|
||||
public void testDoesntDnsResolveLogicalURI() throws Exception {
|
||||
NameService spyNS = spyOnNameService();
|
||||
|
||||
FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
|
||||
String logicalHost = fs.getUri().getHost();
|
||||
Path qualifiedRoot = fs.makeQualified(new Path("/"));
|
||||
|
||||
// Make a few calls against the filesystem.
|
||||
fs.getCanonicalServiceName();
|
||||
fs.listStatus(qualifiedRoot);
|
||||
|
||||
// Ensure that the logical hostname was never resolved.
|
||||
Mockito.verify(spyNS, Mockito.never()).lookupAllHostAddr(Mockito.eq(logicalHost));
|
||||
}
|
||||
|
||||
/**
|
||||
* Same test as above, but for FileContext.
|
||||
*/
|
||||
@Test
|
||||
public void testFileContextDoesntDnsResolveLogicalURI() throws Exception {
|
||||
NameService spyNS = spyOnNameService();
|
||||
FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
|
||||
String logicalHost = fs.getUri().getHost();
|
||||
Configuration haClientConf = fs.getConf();
|
||||
|
||||
FileContext fc = FileContext.getFileContext(haClientConf);
|
||||
Path root = new Path("/");
|
||||
fc.listStatus(root);
|
||||
fc.listStatus(fc.makeQualified(root));
|
||||
fc.getDefaultFileSystem().getCanonicalServiceName();
|
||||
|
||||
// Ensure that the logical hostname was never resolved.
|
||||
Mockito.verify(spyNS, Mockito.never()).lookupAllHostAddr(Mockito.eq(logicalHost));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,6 +35,7 @@ import java.util.Arrays;
|
|||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.Scanner;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.zip.DeflaterOutputStream;
|
||||
import java.util.zip.GZIPOutputStream;
|
||||
|
||||
|
@ -68,7 +69,8 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERV
|
|||
*/
|
||||
public class TestDFSShell {
|
||||
private static final Log LOG = LogFactory.getLog(TestDFSShell.class);
|
||||
|
||||
private static AtomicInteger counter = new AtomicInteger();
|
||||
|
||||
static final String TEST_ROOT_DIR =
|
||||
new Path(System.getProperty("test.build.data","/tmp"))
|
||||
.toString().replace(' ', '+');
|
||||
|
@ -103,7 +105,7 @@ public class TestDFSShell {
|
|||
System.out.println(Thread.currentThread().getStackTrace()[2] + " " + s);
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testZeroSizeFile() throws IOException {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
||||
|
@ -146,7 +148,7 @@ public class TestDFSShell {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testRecrusiveRm() throws IOException {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
||||
|
@ -172,7 +174,7 @@ public class TestDFSShell {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testDu() throws IOException {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
||||
|
@ -222,7 +224,8 @@ public class TestDFSShell {
|
|||
}
|
||||
|
||||
}
|
||||
@Test
|
||||
|
||||
@Test (timeout = 30000)
|
||||
public void testPut() throws IOException {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
||||
|
@ -321,7 +324,7 @@ public class TestDFSShell {
|
|||
|
||||
|
||||
/** check command error outputs and exit statuses. */
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testErrOutPut() throws Exception {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
MiniDFSCluster cluster = null;
|
||||
|
@ -471,7 +474,7 @@ public class TestDFSShell {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testURIPaths() throws Exception {
|
||||
Configuration srcConf = new HdfsConfiguration();
|
||||
Configuration dstConf = new HdfsConfiguration();
|
||||
|
@ -511,7 +514,7 @@ public class TestDFSShell {
|
|||
createLocalFile(furi);
|
||||
argv = new String[3];
|
||||
argv[0] = "-put";
|
||||
argv[1] = furi.toString();
|
||||
argv[1] = furi.toURI().toString();
|
||||
argv[2] = dstFs.getUri().toString() + "/furi";
|
||||
ret = ToolRunner.run(shell, argv);
|
||||
assertEquals(" put is working ", 0, ret);
|
||||
|
@ -564,7 +567,7 @@ public class TestDFSShell {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testText() throws Exception {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
MiniDFSCluster cluster = null;
|
||||
|
@ -680,7 +683,7 @@ public class TestDFSShell {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testCopyToLocal() throws IOException {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
||||
|
@ -778,7 +781,7 @@ public class TestDFSShell {
|
|||
return path;
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testCount() throws Exception {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
||||
|
@ -866,52 +869,59 @@ public class TestDFSShell {
|
|||
shell.setConf(conf);
|
||||
|
||||
try {
|
||||
//first make dir
|
||||
Path dir = new Path(chmodDir);
|
||||
fs.delete(dir, true);
|
||||
fs.mkdirs(dir);
|
||||
//first make dir
|
||||
Path dir = new Path(chmodDir);
|
||||
fs.delete(dir, true);
|
||||
fs.mkdirs(dir);
|
||||
|
||||
confirmPermissionChange(/* Setting */ "u+rwx,g=rw,o-rwx",
|
||||
confirmPermissionChange(/* Setting */ "u+rwx,g=rw,o-rwx",
|
||||
/* Should give */ "rwxrw----", fs, shell, dir);
|
||||
|
||||
//create an empty file
|
||||
Path file = new Path(chmodDir, "file");
|
||||
TestDFSShell.writeFile(fs, file);
|
||||
|
||||
//test octal mode
|
||||
confirmPermissionChange( "644", "rw-r--r--", fs, shell, file);
|
||||
//create an empty file
|
||||
Path file = new Path(chmodDir, "file");
|
||||
TestDFSShell.writeFile(fs, file);
|
||||
|
||||
//test recursive
|
||||
runCmd(shell, "-chmod", "-R", "a+rwX", chmodDir);
|
||||
assertEquals("rwxrwxrwx",
|
||||
fs.getFileStatus(dir).getPermission().toString());
|
||||
assertEquals("rw-rw-rw-",
|
||||
fs.getFileStatus(file).getPermission().toString());
|
||||
//test octal mode
|
||||
confirmPermissionChange("644", "rw-r--r--", fs, shell, file);
|
||||
|
||||
// test sticky bit on directories
|
||||
Path dir2 = new Path(dir, "stickybit" );
|
||||
fs.mkdirs(dir2 );
|
||||
LOG.info("Testing sticky bit on: " + dir2);
|
||||
LOG.info("Sticky bit directory initial mode: " +
|
||||
fs.getFileStatus(dir2).getPermission());
|
||||
|
||||
confirmPermissionChange("u=rwx,g=rx,o=rx", "rwxr-xr-x", fs, shell, dir2);
|
||||
|
||||
confirmPermissionChange("+t", "rwxr-xr-t", fs, shell, dir2);
|
||||
//test recursive
|
||||
runCmd(shell, "-chmod", "-R", "a+rwX", chmodDir);
|
||||
assertEquals("rwxrwxrwx",
|
||||
fs.getFileStatus(dir).getPermission().toString());
|
||||
assertEquals("rw-rw-rw-",
|
||||
fs.getFileStatus(file).getPermission().toString());
|
||||
|
||||
confirmPermissionChange("-t", "rwxr-xr-x", fs, shell, dir2);
|
||||
// Skip "sticky bit" tests on Windows.
|
||||
//
|
||||
if (!Path.WINDOWS) {
|
||||
// test sticky bit on directories
|
||||
Path dir2 = new Path(dir, "stickybit");
|
||||
fs.mkdirs(dir2);
|
||||
LOG.info("Testing sticky bit on: " + dir2);
|
||||
LOG.info("Sticky bit directory initial mode: " +
|
||||
fs.getFileStatus(dir2).getPermission());
|
||||
|
||||
confirmPermissionChange("=t", "--------T", fs, shell, dir2);
|
||||
confirmPermissionChange("u=rwx,g=rx,o=rx", "rwxr-xr-x", fs, shell, dir2);
|
||||
|
||||
confirmPermissionChange("0000", "---------", fs, shell, dir2);
|
||||
confirmPermissionChange("+t", "rwxr-xr-t", fs, shell, dir2);
|
||||
|
||||
confirmPermissionChange("1666", "rw-rw-rwT", fs, shell, dir2);
|
||||
confirmPermissionChange("-t", "rwxr-xr-x", fs, shell, dir2);
|
||||
|
||||
confirmPermissionChange("=t", "--------T", fs, shell, dir2);
|
||||
|
||||
confirmPermissionChange("0000", "---------", fs, shell, dir2);
|
||||
|
||||
confirmPermissionChange("1666", "rw-rw-rwT", fs, shell, dir2);
|
||||
|
||||
confirmPermissionChange("777", "rwxrwxrwt", fs, shell, dir2);
|
||||
|
||||
fs.delete(dir2, true);
|
||||
} else {
|
||||
LOG.info("Skipped sticky bit tests on Windows");
|
||||
}
|
||||
|
||||
fs.delete(dir, true);
|
||||
|
||||
confirmPermissionChange("777", "rwxrwxrwt", fs, shell, dir2);
|
||||
|
||||
fs.delete(dir2, true);
|
||||
fs.delete(dir, true);
|
||||
|
||||
} finally {
|
||||
try {
|
||||
fs.close();
|
||||
|
@ -945,7 +955,7 @@ public class TestDFSShell {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testFilePermissions() throws IOException {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
|
||||
|
@ -1011,7 +1021,7 @@ public class TestDFSShell {
|
|||
/**
|
||||
* Tests various options of DFSShell.
|
||||
*/
|
||||
@Test
|
||||
@Test (timeout = 120000)
|
||||
public void testDFSShell() throws IOException {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
/* This tests some properties of ChecksumFileSystem as well.
|
||||
|
@ -1391,7 +1401,7 @@ public class TestDFSShell {
|
|||
String run(int exitcode, String... options) throws IOException;
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testRemoteException() throws Exception {
|
||||
UserGroupInformation tmpUGI =
|
||||
UserGroupInformation.createUserForTesting("tmpname", new String[] {"mygroup"});
|
||||
|
@ -1435,73 +1445,96 @@ public class TestDFSShell {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testGet() throws IOException {
|
||||
DFSTestUtil.setLogLevel2All(FSInputChecker.LOG);
|
||||
|
||||
final String fname = "testGet.txt";
|
||||
Path root = new Path("/test/get");
|
||||
final Path remotef = new Path(root, fname);
|
||||
final Configuration conf = new HdfsConfiguration();
|
||||
// Race can happen here: block scanner is reading the file when test tries
|
||||
// to corrupt the test file, which will fail the test on Windows platform.
|
||||
// Disable block scanner to avoid this race.
|
||||
conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);
|
||||
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
||||
DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
|
||||
|
||||
TestGetRunner runner = new TestGetRunner() {
|
||||
private int count = 0;
|
||||
private FsShell shell = new FsShell(conf);
|
||||
|
||||
public String run(int exitcode, String... options) throws IOException {
|
||||
String dst = TEST_ROOT_DIR + "/" + fname+ ++count;
|
||||
String[] args = new String[options.length + 3];
|
||||
args[0] = "-get";
|
||||
args[args.length - 2] = remotef.toString();
|
||||
args[args.length - 1] = dst;
|
||||
for(int i = 0; i < options.length; i++) {
|
||||
args[i + 1] = options[i];
|
||||
}
|
||||
show("args=" + Arrays.asList(args));
|
||||
|
||||
try {
|
||||
assertEquals(exitcode, shell.run(args));
|
||||
} catch (Exception e) {
|
||||
assertTrue(StringUtils.stringifyException(e), false);
|
||||
}
|
||||
return exitcode == 0? DFSTestUtil.readFile(new File(dst)): null;
|
||||
}
|
||||
};
|
||||
|
||||
File localf = createLocalFile(new File(TEST_ROOT_DIR, fname));
|
||||
MiniDFSCluster cluster = null;
|
||||
DistributedFileSystem dfs = null;
|
||||
|
||||
try {
|
||||
final String fname = "testGet.txt";
|
||||
final File localf = createLocalFile(new File(TEST_ROOT_DIR, fname));
|
||||
final String localfcontent = DFSTestUtil.readFile(localf);
|
||||
final Path root = mkdir(dfs, new Path("/test/get"));
|
||||
final Path remotef = new Path(root, fname);
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).format(true)
|
||||
.build();
|
||||
dfs = (DistributedFileSystem)cluster.getFileSystem();
|
||||
|
||||
mkdir(dfs, root);
|
||||
dfs.copyFromLocalFile(false, false, new Path(localf.getPath()), remotef);
|
||||
|
||||
final FsShell shell = new FsShell();
|
||||
shell.setConf(conf);
|
||||
TestGetRunner runner = new TestGetRunner() {
|
||||
private int count = 0;
|
||||
|
||||
@Override
|
||||
public String run(int exitcode, String... options) throws IOException {
|
||||
String dst = TEST_ROOT_DIR + "/" + fname+ ++count;
|
||||
String[] args = new String[options.length + 3];
|
||||
args[0] = "-get";
|
||||
args[args.length - 2] = remotef.toString();
|
||||
args[args.length - 1] = dst;
|
||||
for(int i = 0; i < options.length; i++) {
|
||||
args[i + 1] = options[i];
|
||||
}
|
||||
show("args=" + Arrays.asList(args));
|
||||
|
||||
try {
|
||||
assertEquals(exitcode, shell.run(args));
|
||||
} catch (Exception e) {
|
||||
assertTrue(StringUtils.stringifyException(e), false);
|
||||
}
|
||||
return exitcode == 0? DFSTestUtil.readFile(new File(dst)): null;
|
||||
}
|
||||
};
|
||||
String localfcontent = DFSTestUtil.readFile(localf);
|
||||
|
||||
assertEquals(localfcontent, runner.run(0));
|
||||
assertEquals(localfcontent, runner.run(0, "-ignoreCrc"));
|
||||
|
||||
//find and modify the block files
|
||||
// find block files to modify later
|
||||
List<File> files = getBlockFiles(cluster);
|
||||
|
||||
// Shut down cluster and then corrupt the block files by overwriting a
|
||||
// portion with junk data. We must shut down the cluster so that threads
|
||||
// in the data node do not hold locks on the block files while we try to
|
||||
// write into them. Particularly on Windows, the data node's use of the
|
||||
// FileChannel.transferTo method can cause block files to be memory mapped
|
||||
// in read-only mode during the transfer to a client, and this causes a
|
||||
// locking conflict. The call to shutdown the cluster blocks until all
|
||||
// DataXceiver threads exit, preventing this problem.
|
||||
dfs.close();
|
||||
cluster.shutdown();
|
||||
|
||||
show("files=" + files);
|
||||
corrupt(files);
|
||||
|
||||
// Start the cluster again, but do not reformat, so prior files remain.
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).format(false)
|
||||
.build();
|
||||
dfs = (DistributedFileSystem)cluster.getFileSystem();
|
||||
|
||||
assertEquals(null, runner.run(1));
|
||||
String corruptedcontent = runner.run(0, "-ignoreCrc");
|
||||
assertEquals(localfcontent.substring(1), corruptedcontent.substring(1));
|
||||
assertEquals(localfcontent.charAt(0)+1, corruptedcontent.charAt(0));
|
||||
|
||||
localf.delete();
|
||||
} finally {
|
||||
try {dfs.close();} catch (Exception e) {}
|
||||
cluster.shutdown();
|
||||
if (null != dfs) {
|
||||
try {
|
||||
dfs.close();
|
||||
} catch (Exception e) {
|
||||
}
|
||||
}
|
||||
if (null != cluster) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
localf.delete();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testLsr() throws Exception {
|
||||
final Configuration conf = new HdfsConfiguration();
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
||||
|
@ -1559,7 +1592,7 @@ public class TestDFSShell {
|
|||
* and return -1 exit code.
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testInvalidShell() throws Exception {
|
||||
Configuration conf = new Configuration(); // default FS (non-DFS)
|
||||
DFSAdmin admin = new DFSAdmin();
|
||||
|
@ -1569,29 +1602,31 @@ public class TestDFSShell {
|
|||
}
|
||||
|
||||
// force Copy Option is -f
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testCopyCommandsWithForceOption() throws Exception {
|
||||
final int SUCCESS = 0;
|
||||
final int ERROR = 1;
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
|
||||
.format(true).build();
|
||||
FsShell shell = null;
|
||||
FileSystem fs = null;
|
||||
final File localFile = new File(TEST_ROOT_DIR, "testFileForPut");
|
||||
final String localfilepath = localFile.getAbsolutePath();
|
||||
final String testdir = TEST_ROOT_DIR + "/ForceTestDir";
|
||||
final String localfilepath = new Path(localFile.getAbsolutePath()).toUri().toString();
|
||||
final String testdir = "/tmp/TestDFSShell-testCopyCommandsWithForceOption-"
|
||||
+ counter.getAndIncrement();
|
||||
final Path hdfsTestDir = new Path(testdir);
|
||||
try {
|
||||
fs = cluster.getFileSystem();
|
||||
fs.mkdirs(hdfsTestDir);
|
||||
localFile.createNewFile();
|
||||
writeFile(fs, new Path(TEST_ROOT_DIR, "testFileForPut"));
|
||||
writeFile(fs, new Path(testdir, "testFileForPut"));
|
||||
shell = new FsShell();
|
||||
|
||||
// Tests for put
|
||||
String[] argv = new String[] { "-put", "-f", localfilepath, testdir };
|
||||
int res = ToolRunner.run(shell, argv);
|
||||
int SUCCESS = 0;
|
||||
int ERROR = 1;
|
||||
assertEquals("put -f is not working", SUCCESS, res);
|
||||
|
||||
argv = new String[] { "-put", localfilepath, testdir };
|
||||
|
@ -1663,8 +1698,13 @@ public class TestDFSShell {
|
|||
try {
|
||||
// Create and delete a file
|
||||
fs = cluster.getFileSystem();
|
||||
writeFile(fs, new Path(TEST_ROOT_DIR, "foo"));
|
||||
final String testFile = TEST_ROOT_DIR + "/foo";
|
||||
|
||||
// Use a separate tmp dir for each invocation.
|
||||
final String testdir = "/tmp/TestDFSShell-deleteFileUsingTrash-" +
|
||||
counter.getAndIncrement();
|
||||
|
||||
writeFile(fs, new Path(testdir, "foo"));
|
||||
final String testFile = testdir + "/foo";
|
||||
final String trashFile = shell.getCurrentTrashDir() + "/" + testFile;
|
||||
String[] argv = new String[] { "-rm", testFile };
|
||||
int res = ToolRunner.run(shell, argv);
|
||||
|
@ -1696,7 +1736,7 @@ public class TestDFSShell {
|
|||
* Test that the server trash configuration is respected when
|
||||
* the client configuration is not set.
|
||||
*/
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testServerConfigRespected() throws Exception {
|
||||
deleteFileUsingTrash(true, false);
|
||||
}
|
||||
|
@ -1705,7 +1745,7 @@ public class TestDFSShell {
|
|||
* Test that server trash configuration is respected even when the
|
||||
* client configuration is set.
|
||||
*/
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testServerConfigRespectedWithClient() throws Exception {
|
||||
deleteFileUsingTrash(true, true);
|
||||
}
|
||||
|
@ -1714,7 +1754,7 @@ public class TestDFSShell {
|
|||
* Test that the client trash configuration is respected when
|
||||
* the server configuration is not set.
|
||||
*/
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testClientConfigRespected() throws Exception {
|
||||
deleteFileUsingTrash(false, true);
|
||||
}
|
||||
|
@ -1722,7 +1762,7 @@ public class TestDFSShell {
|
|||
/**
|
||||
* Test that trash is disabled by default.
|
||||
*/
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testNoTrashConfig() throws Exception {
|
||||
deleteFileUsingTrash(false, false);
|
||||
}
|
||||
|
|
|
@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
|
@ -293,6 +294,11 @@ public class TestDFSUpgradeFromImage {
|
|||
new File(baseDir, "name2/current/VERSION"),
|
||||
"imageMD5Digest", "22222222222222222222222222222222");
|
||||
|
||||
// Attach our own log appender so we can verify output
|
||||
final LogVerificationAppender appender = new LogVerificationAppender();
|
||||
final Logger logger = Logger.getRootLogger();
|
||||
logger.addAppender(appender);
|
||||
|
||||
// Upgrade should now fail
|
||||
try {
|
||||
upgradeAndVerify(new MiniDFSCluster.Builder(upgradeConf).
|
||||
|
@ -300,9 +306,12 @@ public class TestDFSUpgradeFromImage {
|
|||
fail("Upgrade did not fail with bad MD5");
|
||||
} catch (IOException ioe) {
|
||||
String msg = StringUtils.stringifyException(ioe);
|
||||
if (!msg.contains("is corrupt with MD5 checksum")) {
|
||||
if (!msg.contains("Failed to load an FSImage file")) {
|
||||
throw ioe;
|
||||
}
|
||||
int md5failures = appender.countExceptionsWithMessage(
|
||||
" is corrupt with MD5 checksum of ");
|
||||
assertEquals("Upgrade did not fail with bad MD5", 1, md5failures);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -641,4 +641,24 @@ public class TestDFSUtil {
|
|||
assertFalse(DFSUtil.isValidName("/foo/:/bar"));
|
||||
assertFalse(DFSUtil.isValidName("/foo:bar"));
|
||||
}
|
||||
|
||||
@Test(timeout=5000)
|
||||
public void testGetSpnegoKeytabKey() {
|
||||
HdfsConfiguration conf = new HdfsConfiguration();
|
||||
String defaultKey = "default.spengo.key";
|
||||
conf.unset(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY);
|
||||
assertEquals("Test spnego key in config is null", defaultKey,
|
||||
DFSUtil.getSpnegoKeytabKey(conf, defaultKey));
|
||||
|
||||
conf.set(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY, "");
|
||||
assertEquals("Test spnego key is empty", defaultKey,
|
||||
DFSUtil.getSpnegoKeytabKey(conf, defaultKey));
|
||||
|
||||
String spengoKey = "spengo.key";
|
||||
conf.set(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY,
|
||||
spengoKey);
|
||||
assertEquals("Test spnego key is NOT null",
|
||||
DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY,
|
||||
DFSUtil.getSpnegoKeytabKey(conf, defaultKey));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -71,6 +71,7 @@ public class TestDataTransferKeepalive {
|
|||
.numDataNodes(1).build();
|
||||
fs = cluster.getFileSystem();
|
||||
dfsClient = ((DistributedFileSystem)fs).dfs;
|
||||
dfsClient.peerCache.clear();
|
||||
|
||||
String poolId = cluster.getNamesystem().getBlockPoolId();
|
||||
dn = cluster.getDataNodes().get(0);
|
||||
|
|
|
@ -151,7 +151,7 @@ public class TestFileConcurrentReader {
|
|||
/**
|
||||
* Test that that writes to an incomplete block are available to a reader
|
||||
*/
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testUnfinishedBlockRead()
|
||||
throws IOException {
|
||||
// create a new file in the root, write data, do no close
|
||||
|
@ -174,7 +174,7 @@ public class TestFileConcurrentReader {
|
|||
* would result in too small a buffer to do the buffer-copy needed
|
||||
* for partial chunks.
|
||||
*/
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testUnfinishedBlockPacketBufferOverrun() throws IOException {
|
||||
// check that / exists
|
||||
Path path = new Path("/");
|
||||
|
@ -200,7 +200,7 @@ public class TestFileConcurrentReader {
|
|||
// use a small block size and a large write so that DN is busy creating
|
||||
// new blocks. This makes it almost 100% sure we can reproduce
|
||||
// case of client getting a DN that hasn't yet created the blocks
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testImmediateReadOfNewFile()
|
||||
throws IOException {
|
||||
final int blockSize = 64 * 1024;
|
||||
|
@ -277,12 +277,12 @@ public class TestFileConcurrentReader {
|
|||
|
||||
// for some reason, using tranferTo evokes the race condition more often
|
||||
// so test separately
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testUnfinishedBlockCRCErrorTransferTo() throws IOException {
|
||||
runTestUnfinishedBlockCRCError(true, SyncType.SYNC, DEFAULT_WRITE_SIZE);
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testUnfinishedBlockCRCErrorTransferToVerySmallWrite()
|
||||
throws IOException {
|
||||
runTestUnfinishedBlockCRCError(true, SyncType.SYNC, SMALL_WRITE_SIZE);
|
||||
|
@ -290,18 +290,17 @@ public class TestFileConcurrentReader {
|
|||
|
||||
// fails due to issue w/append, disable
|
||||
@Ignore
|
||||
@Test
|
||||
public void _testUnfinishedBlockCRCErrorTransferToAppend()
|
||||
throws IOException {
|
||||
runTestUnfinishedBlockCRCError(true, SyncType.APPEND, DEFAULT_WRITE_SIZE);
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testUnfinishedBlockCRCErrorNormalTransfer() throws IOException {
|
||||
runTestUnfinishedBlockCRCError(false, SyncType.SYNC, DEFAULT_WRITE_SIZE);
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testUnfinishedBlockCRCErrorNormalTransferVerySmallWrite()
|
||||
throws IOException {
|
||||
runTestUnfinishedBlockCRCError(false, SyncType.SYNC, SMALL_WRITE_SIZE);
|
||||
|
@ -309,7 +308,6 @@ public class TestFileConcurrentReader {
|
|||
|
||||
// fails due to issue w/append, disable
|
||||
@Ignore
|
||||
@Test
|
||||
public void _testUnfinishedBlockCRCErrorNormalTransferAppend()
|
||||
throws IOException {
|
||||
runTestUnfinishedBlockCRCError(false, SyncType.APPEND, DEFAULT_WRITE_SIZE);
|
||||
|
@ -338,33 +336,33 @@ public class TestFileConcurrentReader {
|
|||
final AtomicBoolean writerDone = new AtomicBoolean(false);
|
||||
final AtomicBoolean writerStarted = new AtomicBoolean(false);
|
||||
final AtomicBoolean error = new AtomicBoolean(false);
|
||||
final FSDataOutputStream initialOutputStream = fileSystem.create(file);
|
||||
final Thread writer = new Thread(new Runnable() {
|
||||
private FSDataOutputStream outputStream = initialOutputStream;
|
||||
|
||||
final Thread writer = new Thread(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
for (int i = 0; !error.get() && i < numWrites; i++) {
|
||||
try {
|
||||
FSDataOutputStream outputStream = fileSystem.create(file);
|
||||
if (syncType == SyncType.APPEND) {
|
||||
outputStream.close();
|
||||
outputStream = fileSystem.append(file);
|
||||
}
|
||||
try {
|
||||
for (int i = 0; !error.get() && i < numWrites; i++) {
|
||||
final byte[] writeBuf =
|
||||
DFSTestUtil.generateSequentialBytes(i * writeSize, writeSize);
|
||||
DFSTestUtil.generateSequentialBytes(i * writeSize, writeSize);
|
||||
outputStream.write(writeBuf);
|
||||
if (syncType == SyncType.SYNC) {
|
||||
outputStream.hflush();
|
||||
} else { // append
|
||||
outputStream.close();
|
||||
outputStream = fileSystem.append(file);
|
||||
}
|
||||
writerStarted.set(true);
|
||||
} catch (IOException e) {
|
||||
error.set(true);
|
||||
LOG.error("error writing to file", e);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
error.set(true);
|
||||
LOG.error("error writing to file", e);
|
||||
} finally {
|
||||
outputStream.close();
|
||||
}
|
||||
|
||||
writerDone.set(true);
|
||||
outputStream.close();
|
||||
} catch (Exception e) {
|
||||
LOG.error("error in writer", e);
|
||||
|
||||
|
@ -415,7 +413,6 @@ public class TestFileConcurrentReader {
|
|||
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
initialOutputStream.close();
|
||||
}
|
||||
|
||||
private boolean validateSequentialBytes(byte[] buf, int startPos, int len) {
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -39,7 +40,7 @@ import org.junit.Test;
|
|||
public class TestHftpURLTimeouts {
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
URLUtils.SOCKET_TIMEOUT = 1;
|
||||
URLUtils.SOCKET_TIMEOUT = 5;
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -116,6 +117,7 @@ public class TestHftpURLTimeouts {
|
|||
conns.add(fs.openConnection("/", ""));
|
||||
} catch (SocketTimeoutException ste) {
|
||||
String message = ste.getMessage();
|
||||
assertNotNull(message);
|
||||
// https will get a read timeout due to SSL negotiation, but
|
||||
// a normal http will not, so need to ignore SSL read timeouts
|
||||
// until a connect timeout occurs
|
||||
|
|
|
@ -65,7 +65,7 @@ public class TestMiniDFSCluster {
|
|||
*
|
||||
* @throws Throwable on a failure
|
||||
*/
|
||||
@Test
|
||||
@Test(timeout=100000)
|
||||
public void testClusterWithoutSystemProperties() throws Throwable {
|
||||
System.clearProperty(MiniDFSCluster.PROP_TEST_BUILD_DATA);
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
|
@ -74,7 +74,8 @@ public class TestMiniDFSCluster {
|
|||
conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, c1Path);
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
|
||||
try {
|
||||
assertEquals(c1Path+"/data", cluster.getDataDirectory());
|
||||
assertEquals(new File(c1Path + "/data"),
|
||||
new File(cluster.getDataDirectory()));
|
||||
} finally {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
@ -84,7 +85,7 @@ public class TestMiniDFSCluster {
|
|||
* Bring up two clusters and assert that they are in different directories.
|
||||
* @throws Throwable on a failure
|
||||
*/
|
||||
@Test
|
||||
@Test(timeout=100000)
|
||||
public void testDualClusters() throws Throwable {
|
||||
File testDataCluster2 = new File(testDataPath, CLUSTER_2);
|
||||
File testDataCluster3 = new File(testDataPath, CLUSTER_3);
|
||||
|
@ -95,7 +96,7 @@ public class TestMiniDFSCluster {
|
|||
MiniDFSCluster cluster3 = null;
|
||||
try {
|
||||
String dataDir2 = cluster2.getDataDirectory();
|
||||
assertEquals(c2Path + "/data", dataDir2);
|
||||
assertEquals(new File(c2Path + "/data"), new File(dataDir2));
|
||||
//change the data dir
|
||||
conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR,
|
||||
testDataCluster3.getAbsolutePath());
|
||||
|
|
|
@ -18,12 +18,15 @@
|
|||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.net.URI;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -33,6 +36,7 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||
|
@ -42,6 +46,7 @@ import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
|
|||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
@ -108,9 +113,22 @@ public class TestShortCircuitLocalRead {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static String getCurrentUser() throws IOException {
|
||||
return UserGroupInformation.getCurrentUser().getShortUserName();
|
||||
}
|
||||
|
||||
static void checkFileContent(FileSystem fs, Path name, byte[] expected,
|
||||
int readOffset) throws IOException {
|
||||
/** Check file content, reading as user {@code readingUser} */
|
||||
static void checkFileContent(URI uri, Path name, byte[] expected,
|
||||
int readOffset, String readingUser, Configuration conf,
|
||||
boolean legacyShortCircuitFails)
|
||||
throws IOException, InterruptedException {
|
||||
// Ensure short circuit is enabled
|
||||
DistributedFileSystem fs = getFileSystem(readingUser, uri, conf);
|
||||
if (legacyShortCircuitFails) {
|
||||
assertTrue(fs.getClient().useLegacyBlockReaderLocal());
|
||||
}
|
||||
|
||||
FSDataInputStream stm = fs.open(name);
|
||||
byte[] actual = new byte[expected.length-readOffset];
|
||||
stm.readFully(readOffset, actual);
|
||||
|
@ -135,6 +153,10 @@ public class TestShortCircuitLocalRead {
|
|||
nread += nbytes;
|
||||
}
|
||||
checkData(actual, readOffset, expected, "Read 3");
|
||||
|
||||
if (legacyShortCircuitFails) {
|
||||
assertFalse(fs.getClient().useLegacyBlockReaderLocal());
|
||||
}
|
||||
stm.close();
|
||||
}
|
||||
|
||||
|
@ -146,11 +168,17 @@ public class TestShortCircuitLocalRead {
|
|||
return arr;
|
||||
}
|
||||
|
||||
/**
|
||||
* Verifies that reading a file with the direct read(ByteBuffer) api gives the expected set of bytes.
|
||||
*/
|
||||
static void checkFileContentDirect(FileSystem fs, Path name, byte[] expected,
|
||||
int readOffset) throws IOException {
|
||||
/** Check the file content, reading as user {@code readingUser} */
|
||||
static void checkFileContentDirect(URI uri, Path name, byte[] expected,
|
||||
int readOffset, String readingUser, Configuration conf,
|
||||
boolean legacyShortCircuitFails)
|
||||
throws IOException, InterruptedException {
|
||||
// Ensure short circuit is enabled
|
||||
DistributedFileSystem fs = getFileSystem(readingUser, uri, conf);
|
||||
if (legacyShortCircuitFails) {
|
||||
assertTrue(fs.getClient().useLegacyBlockReaderLocal());
|
||||
}
|
||||
|
||||
HdfsDataInputStream stm = (HdfsDataInputStream)fs.open(name);
|
||||
|
||||
ByteBuffer actual = ByteBuffer.allocateDirect(expected.length - readOffset);
|
||||
|
@ -180,15 +208,33 @@ public class TestShortCircuitLocalRead {
|
|||
nread += nbytes;
|
||||
}
|
||||
checkData(arrayFromByteBuffer(actual), readOffset, expected, "Read 3");
|
||||
if (legacyShortCircuitFails) {
|
||||
assertFalse(fs.getClient().useLegacyBlockReaderLocal());
|
||||
}
|
||||
stm.close();
|
||||
}
|
||||
|
||||
public void doTestShortCircuitReadLegacy(boolean ignoreChecksum, int size,
|
||||
int readOffset, String shortCircuitUser, String readingUser,
|
||||
boolean legacyShortCircuitFails) throws IOException, InterruptedException {
|
||||
doTestShortCircuitReadImpl(ignoreChecksum, size, readOffset,
|
||||
shortCircuitUser, readingUser, legacyShortCircuitFails);
|
||||
}
|
||||
|
||||
public void doTestShortCircuitRead(boolean ignoreChecksum, int size,
|
||||
int readOffset) throws IOException, InterruptedException {
|
||||
String shortCircuitUser = getCurrentUser();
|
||||
doTestShortCircuitReadImpl(ignoreChecksum, size, readOffset,
|
||||
null, getCurrentUser(), false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that file data can be read by reading the block file
|
||||
* directly from the local store.
|
||||
*/
|
||||
public void doTestShortCircuitRead(boolean ignoreChecksum, int size,
|
||||
int readOffset) throws IOException {
|
||||
public void doTestShortCircuitReadImpl(boolean ignoreChecksum, int size,
|
||||
int readOffset, String shortCircuitUser, String readingUser,
|
||||
boolean legacyShortCircuitFails) throws IOException, InterruptedException {
|
||||
Configuration conf = new Configuration();
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
|
||||
|
@ -196,6 +242,11 @@ public class TestShortCircuitLocalRead {
|
|||
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
|
||||
new File(sockDir.getDir(),
|
||||
"TestShortCircuitLocalRead._PORT.sock").getAbsolutePath());
|
||||
if (shortCircuitUser != null) {
|
||||
conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
|
||||
shortCircuitUser);
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL, true);
|
||||
}
|
||||
if (simulatedStorage) {
|
||||
SimulatedFSDataset.setFactory(conf);
|
||||
}
|
||||
|
@ -208,53 +259,94 @@ public class TestShortCircuitLocalRead {
|
|||
assertTrue("/ should be a directory", fs.getFileStatus(path)
|
||||
.isDirectory() == true);
|
||||
|
||||
byte[] fileData = AppendTestUtil.randomBytes(seed, size);
|
||||
// create a new file in home directory. Do not close it.
|
||||
Path file1 = new Path("filelocal.dat");
|
||||
byte[] fileData = AppendTestUtil.randomBytes(seed, size);
|
||||
Path file1 = fs.makeQualified(new Path("filelocal.dat"));
|
||||
FSDataOutputStream stm = createFile(fs, file1, 1);
|
||||
|
||||
// write to file
|
||||
stm.write(fileData);
|
||||
stm.close();
|
||||
checkFileContent(fs, file1, fileData, readOffset);
|
||||
checkFileContentDirect(fs, file1, fileData, readOffset);
|
||||
|
||||
URI uri = cluster.getURI();
|
||||
checkFileContent(uri, file1, fileData, readOffset, readingUser, conf,
|
||||
legacyShortCircuitFails);
|
||||
checkFileContentDirect(uri, file1, fileData, readOffset, readingUser,
|
||||
conf, legacyShortCircuitFails);
|
||||
} finally {
|
||||
fs.close();
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFileLocalReadNoChecksum() throws IOException {
|
||||
@Test(timeout=10000)
|
||||
public void testFileLocalReadNoChecksum() throws Exception {
|
||||
doTestShortCircuitRead(true, 3*blockSize+100, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFileLocalReadChecksum() throws IOException {
|
||||
@Test(timeout=10000)
|
||||
public void testFileLocalReadChecksum() throws Exception {
|
||||
doTestShortCircuitRead(false, 3*blockSize+100, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSmallFileLocalRead() throws IOException {
|
||||
@Test(timeout=10000)
|
||||
public void testSmallFileLocalRead() throws Exception {
|
||||
doTestShortCircuitRead(false, 13, 0);
|
||||
doTestShortCircuitRead(false, 13, 5);
|
||||
doTestShortCircuitRead(true, 13, 0);
|
||||
doTestShortCircuitRead(true, 13, 5);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadFromAnOffset() throws IOException {
|
||||
@Test(timeout=10000)
|
||||
public void testLocalReadLegacy() throws Exception {
|
||||
doTestShortCircuitReadLegacy(true, 13, 0, getCurrentUser(),
|
||||
getCurrentUser(), false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Try a short circuit from a reader that is not allowed to
|
||||
* to use short circuit. The test ensures reader falls back to non
|
||||
* shortcircuit reads when shortcircuit is disallowed.
|
||||
*/
|
||||
@Test(timeout=10000)
|
||||
public void testLocalReadFallback() throws Exception {
|
||||
doTestShortCircuitReadLegacy(true, 13, 0, getCurrentUser(), "notallowed", true);
|
||||
}
|
||||
|
||||
@Test(timeout=10000)
|
||||
public void testReadFromAnOffset() throws Exception {
|
||||
doTestShortCircuitRead(false, 3*blockSize+100, 777);
|
||||
doTestShortCircuitRead(true, 3*blockSize+100, 777);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLongFile() throws IOException {
|
||||
@Test(timeout=10000)
|
||||
public void testLongFile() throws Exception {
|
||||
doTestShortCircuitRead(false, 10*blockSize+100, 777);
|
||||
doTestShortCircuitRead(true, 10*blockSize+100, 777);
|
||||
}
|
||||
|
||||
@Test
|
||||
private ClientDatanodeProtocol getProxy(UserGroupInformation ugi,
|
||||
final DatanodeID dnInfo, final Configuration conf) throws IOException,
|
||||
InterruptedException {
|
||||
return ugi.doAs(new PrivilegedExceptionAction<ClientDatanodeProtocol>() {
|
||||
@Override
|
||||
public ClientDatanodeProtocol run() throws Exception {
|
||||
return DFSUtil.createClientDatanodeProtocolProxy(dnInfo, conf, 60000,
|
||||
false);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private static DistributedFileSystem getFileSystem(String user, final URI uri,
|
||||
final Configuration conf) throws InterruptedException, IOException {
|
||||
UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
|
||||
return ugi.doAs(new PrivilegedExceptionAction<DistributedFileSystem>() {
|
||||
@Override
|
||||
public DistributedFileSystem run() throws Exception {
|
||||
return (DistributedFileSystem)FileSystem.get(uri, conf);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Test(timeout=10000)
|
||||
public void testDeprecatedGetBlockLocalPathInfoRpc()
|
||||
throws IOException, InterruptedException {
|
||||
final Configuration conf = new Configuration();
|
||||
|
@ -287,7 +379,7 @@ public class TestShortCircuitLocalRead {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test(timeout=10000)
|
||||
public void testSkipWithVerifyChecksum() throws IOException {
|
||||
int size = blockSize;
|
||||
Configuration conf = new Configuration();
|
||||
|
@ -417,7 +509,7 @@ public class TestShortCircuitLocalRead {
|
|||
}
|
||||
|
||||
/**
|
||||
* Test to run benchmarks between shortcircuit read vs regular read with
|
||||
* Test to run benchmarks between short circuit read vs regular read with
|
||||
* specified number of threads simultaneously reading.
|
||||
* <br>
|
||||
* Run this using the following command:
|
||||
|
@ -435,7 +527,7 @@ public class TestShortCircuitLocalRead {
|
|||
int threadCount = Integer.valueOf(args[2]);
|
||||
|
||||
// Setup create a file
|
||||
Configuration conf = new Configuration();
|
||||
final Configuration conf = new Configuration();
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, shortcircuit);
|
||||
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
|
||||
"/tmp/TestShortCircuitLocalRead._PORT");
|
||||
|
@ -463,9 +555,13 @@ public class TestShortCircuitLocalRead {
|
|||
public void run() {
|
||||
for (int i = 0; i < iteration; i++) {
|
||||
try {
|
||||
checkFileContent(fs, file1, dataToWrite, 0);
|
||||
String user = getCurrentUser();
|
||||
checkFileContent(fs.getUri(), file1, dataToWrite, 0, user, conf,
|
||||
true);
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
} catch (InterruptedException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
package org.apache.hadoop.hdfs.qjournal;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
import static org.junit.Assume.*;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
@ -43,7 +44,7 @@ import org.junit.Test;
|
|||
|
||||
public class TestNNWithQJM {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
private MiniJournalCluster mjc;
|
||||
private MiniJournalCluster mjc = null;
|
||||
private Path TEST_PATH = new Path("/test-dir");
|
||||
private Path TEST_PATH_2 = new Path("/test-dir");
|
||||
|
||||
|
@ -61,10 +62,11 @@ public class TestNNWithQJM {
|
|||
public void stopJNs() throws Exception {
|
||||
if (mjc != null) {
|
||||
mjc.shutdown();
|
||||
mjc = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testLogAndRestart() throws IOException {
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
|
||||
MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
|
||||
|
@ -93,9 +95,12 @@ public class TestNNWithQJM {
|
|||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
@Test (timeout = 30000)
|
||||
public void testNewNamenodeTakesOverWriter() throws Exception {
|
||||
// Skip the test on Windows. See HDFS-4584.
|
||||
assumeTrue(!Path.WINDOWS);
|
||||
|
||||
File nn1Dir = new File(
|
||||
MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image-nn1");
|
||||
File nn2Dir = new File(
|
||||
|
@ -154,7 +159,7 @@ public class TestNNWithQJM {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 30000)
|
||||
public void testMismatchedNNIsRejected() throws Exception {
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
|
||||
MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
|
||||
|
@ -188,8 +193,8 @@ public class TestNNWithQJM {
|
|||
"Unable to start log segment 1: too few journals", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
@Test (timeout = 30000)
|
||||
public void testWebPageHasQjmInfo() throws Exception {
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
|
||||
MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
|
||||
|
|
|
@ -36,10 +36,7 @@ import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
|
|||
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.junit.After;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.*;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
public class TestJournal {
|
||||
|
@ -77,7 +74,7 @@ public class TestJournal {
|
|||
IOUtils.closeStream(journal);
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 10000)
|
||||
public void testEpochHandling() throws Exception {
|
||||
assertEquals(0, journal.getLastPromisedEpoch());
|
||||
NewEpochResponseProto newEpoch =
|
||||
|
@ -110,7 +107,7 @@ public class TestJournal {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 10000)
|
||||
public void testMaintainCommittedTxId() throws Exception {
|
||||
journal.newEpoch(FAKE_NSINFO, 1);
|
||||
journal.startLogSegment(makeRI(1), 1);
|
||||
|
@ -125,7 +122,7 @@ public class TestJournal {
|
|||
assertEquals(3, journal.getCommittedTxnIdForTests());
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 10000)
|
||||
public void testRestartJournal() throws Exception {
|
||||
journal.newEpoch(FAKE_NSINFO, 1);
|
||||
journal.startLogSegment(makeRI(1), 1);
|
||||
|
@ -149,7 +146,7 @@ public class TestJournal {
|
|||
assertEquals(1, newEpoch.getLastSegmentTxId());
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 10000)
|
||||
public void testFormatResetsCachedValues() throws Exception {
|
||||
journal.newEpoch(FAKE_NSINFO, 12345L);
|
||||
journal.startLogSegment(new RequestInfo(JID, 12345L, 1L, 0L), 1L);
|
||||
|
@ -158,6 +155,8 @@ public class TestJournal {
|
|||
assertEquals(12345L, journal.getLastWriterEpoch());
|
||||
assertTrue(journal.isFormatted());
|
||||
|
||||
// Close the journal in preparation for reformatting it.
|
||||
journal.close();
|
||||
journal.format(FAKE_NSINFO_2);
|
||||
|
||||
assertEquals(0, journal.getLastPromisedEpoch());
|
||||
|
@ -170,7 +169,7 @@ public class TestJournal {
|
|||
* before any transactions are written, that the next newEpoch() call
|
||||
* returns the prior segment txid as its most recent segment.
|
||||
*/
|
||||
@Test
|
||||
@Test (timeout = 10000)
|
||||
public void testNewEpochAtBeginningOfSegment() throws Exception {
|
||||
journal.newEpoch(FAKE_NSINFO, 1);
|
||||
journal.startLogSegment(makeRI(1), 1);
|
||||
|
@ -182,7 +181,7 @@ public class TestJournal {
|
|||
assertEquals(1, resp.getLastSegmentTxId());
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 10000)
|
||||
public void testJournalLocking() throws Exception {
|
||||
Assume.assumeTrue(journal.getStorage().getStorageDir(0).isLockSupported());
|
||||
StorageDirectory sd = journal.getStorage().getStorageDir(0);
|
||||
|
@ -206,13 +205,14 @@ public class TestJournal {
|
|||
// Hence, should be able to create a new Journal in the same dir.
|
||||
Journal journal2 = new Journal(TEST_LOG_DIR, JID, mockErrorReporter);
|
||||
journal2.newEpoch(FAKE_NSINFO, 2);
|
||||
journal2.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test finalizing a segment after some batch of edits were missed.
|
||||
* This should fail, since we validate the log before finalization.
|
||||
*/
|
||||
@Test
|
||||
@Test (timeout = 10000)
|
||||
public void testFinalizeWhenEditsAreMissed() throws Exception {
|
||||
journal.newEpoch(FAKE_NSINFO, 1);
|
||||
journal.startLogSegment(makeRI(1), 1);
|
||||
|
@ -246,7 +246,7 @@ public class TestJournal {
|
|||
* Ensure that finalizing a segment which doesn't exist throws the
|
||||
* appropriate exception.
|
||||
*/
|
||||
@Test
|
||||
@Test (timeout = 10000)
|
||||
public void testFinalizeMissingSegment() throws Exception {
|
||||
journal.newEpoch(FAKE_NSINFO, 1);
|
||||
try {
|
||||
|
@ -267,7 +267,7 @@ public class TestJournal {
|
|||
* Eventually, the connection comes back, and the NN tries to start a new
|
||||
* segment at a higher txid. This should abort the old one and succeed.
|
||||
*/
|
||||
@Test
|
||||
@Test (timeout = 10000)
|
||||
public void testAbortOldSegmentIfFinalizeIsMissed() throws Exception {
|
||||
journal.newEpoch(FAKE_NSINFO, 1);
|
||||
|
||||
|
@ -296,7 +296,7 @@ public class TestJournal {
|
|||
* Test behavior of startLogSegment() when a segment with the
|
||||
* same transaction ID already exists.
|
||||
*/
|
||||
@Test
|
||||
@Test (timeout = 10000)
|
||||
public void testStartLogSegmentWhenAlreadyExists() throws Exception {
|
||||
journal.newEpoch(FAKE_NSINFO, 1);
|
||||
|
||||
|
@ -345,7 +345,7 @@ public class TestJournal {
|
|||
return new RequestInfo(JID, 1, serial, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test (timeout = 10000)
|
||||
public void testNamespaceVerification() throws Exception {
|
||||
journal.newEpoch(FAKE_NSINFO, 1);
|
||||
|
||||
|
|
|
@ -46,6 +46,7 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
|||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.test.MetricsAsserts;
|
||||
import org.apache.hadoop.util.Shell;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
@ -65,6 +66,8 @@ public class TestJournalNode {
|
|||
private Configuration conf = new Configuration();
|
||||
private IPCLoggerChannel ch;
|
||||
private String journalId;
|
||||
private File TEST_BUILD_DATA =
|
||||
new File(System.getProperty("test.build.data", "build/test/data"));
|
||||
|
||||
static {
|
||||
// Avoid an error when we double-initialize JvmMetrics
|
||||
|
@ -96,7 +99,7 @@ public class TestJournalNode {
|
|||
jn.stop(0);
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test(timeout=100000)
|
||||
public void testJournal() throws Exception {
|
||||
MetricsRecordBuilder metrics = MetricsAsserts.getMetrics(
|
||||
journal.getMetricsForTests().getName());
|
||||
|
@ -129,7 +132,7 @@ public class TestJournalNode {
|
|||
}
|
||||
|
||||
|
||||
@Test
|
||||
@Test(timeout=100000)
|
||||
public void testReturnsSegmentInfoAtEpochTransition() throws Exception {
|
||||
ch.newEpoch(1).get();
|
||||
ch.setEpoch(1);
|
||||
|
@ -157,7 +160,7 @@ public class TestJournalNode {
|
|||
assertEquals(1, response.getLastSegmentTxId());
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test(timeout=100000)
|
||||
public void testHttpServer() throws Exception {
|
||||
InetSocketAddress addr = jn.getBoundHttpAddress();
|
||||
assertTrue(addr.getPort() > 0);
|
||||
|
@ -210,7 +213,7 @@ public class TestJournalNode {
|
|||
* Test that the JournalNode performs correctly as a Paxos
|
||||
* <em>Acceptor</em> process.
|
||||
*/
|
||||
@Test
|
||||
@Test(timeout=100000)
|
||||
public void testAcceptRecoveryBehavior() throws Exception {
|
||||
// We need to run newEpoch() first, or else we have no way to distinguish
|
||||
// different proposals for the same decision.
|
||||
|
@ -270,20 +273,27 @@ public class TestJournalNode {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test(timeout=100000)
|
||||
public void testFailToStartWithBadConfig() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, "non-absolute-path");
|
||||
assertJNFailsToStart(conf, "should be an absolute path");
|
||||
|
||||
// Existing file which is not a directory
|
||||
conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, "/dev/null");
|
||||
assertJNFailsToStart(conf, "is not a directory");
|
||||
File existingFile = new File(TEST_BUILD_DATA, "testjournalnodefile");
|
||||
assertTrue(existingFile.createNewFile());
|
||||
try {
|
||||
conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY,
|
||||
existingFile.getAbsolutePath());
|
||||
assertJNFailsToStart(conf, "Not a directory");
|
||||
} finally {
|
||||
existingFile.delete();
|
||||
}
|
||||
|
||||
// Directory which cannot be created
|
||||
conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, "/proc/does-not-exist");
|
||||
assertJNFailsToStart(conf, "Could not create");
|
||||
|
||||
conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY,
|
||||
Shell.WINDOWS ? "\\\\cannotBeCreated" : "/proc/does-not-exist");
|
||||
assertJNFailsToStart(conf, "Can not create directory");
|
||||
}
|
||||
|
||||
private static void assertJNFailsToStart(Configuration conf,
|
||||
|
|
|
@ -104,7 +104,7 @@ public class TestNodeCount {
|
|||
while (iter.hasNext()) {
|
||||
DatanodeDescriptor dn = iter.next();
|
||||
Collection<Block> blocks = bm.excessReplicateMap.get(dn.getStorageID());
|
||||
if (blocks == null || !blocks.contains(block) ) {
|
||||
if (blocks == null || !blocks.contains(block.getLocalBlock()) ) {
|
||||
nonExcessDN = dn;
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.LogVerificationAppender;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
|
@ -45,7 +46,6 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|||
import org.apache.hadoop.net.NetworkTopology;
|
||||
import org.apache.hadoop.net.Node;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.apache.log4j.AppenderSkeleton;
|
||||
import org.apache.log4j.Level;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.log4j.spi.LoggingEvent;
|
||||
|
@ -419,7 +419,7 @@ public class TestReplicationPolicy {
|
|||
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0);
|
||||
}
|
||||
|
||||
final TestAppender appender = new TestAppender();
|
||||
final LogVerificationAppender appender = new LogVerificationAppender();
|
||||
final Logger logger = Logger.getRootLogger();
|
||||
logger.addAppender(appender);
|
||||
|
||||
|
@ -446,28 +446,6 @@ public class TestReplicationPolicy {
|
|||
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
|
||||
}
|
||||
}
|
||||
|
||||
class TestAppender extends AppenderSkeleton {
|
||||
private final List<LoggingEvent> log = new ArrayList<LoggingEvent>();
|
||||
|
||||
@Override
|
||||
public boolean requiresLayout() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void append(final LoggingEvent loggingEvent) {
|
||||
log.add(loggingEvent);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
}
|
||||
|
||||
public List<LoggingEvent> getLog() {
|
||||
return new ArrayList<LoggingEvent>(log);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean containsWithinRange(DatanodeDescriptor target,
|
||||
DatanodeDescriptor[] nodes, int startIndex, int endIndex) {
|
||||
|
|
|
@ -27,33 +27,26 @@ import java.util.List;
|
|||
import org.junit.Test;
|
||||
import static org.junit.Assert.*;
|
||||
import static org.mockito.Mockito.*;
|
||||
import static org.apache.hadoop.test.MockitoMaker.*;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode.DataNodeDiskChecker;
|
||||
|
||||
public class TestDataDirs {
|
||||
|
||||
@Test public void testGetDataDirsFromURIs() throws Throwable {
|
||||
File localDir = make(stub(File.class).returning(true).from.exists());
|
||||
when(localDir.mkdir()).thenReturn(true);
|
||||
FsPermission normalPerm = new FsPermission("700");
|
||||
FsPermission badPerm = new FsPermission("000");
|
||||
FileStatus stat = make(stub(FileStatus.class)
|
||||
.returning(normalPerm, normalPerm, badPerm).from.getPermission());
|
||||
when(stat.isDirectory()).thenReturn(true);
|
||||
LocalFileSystem fs = make(stub(LocalFileSystem.class)
|
||||
.returning(stat).from.getFileStatus(any(Path.class)));
|
||||
when(fs.pathToFile(any(Path.class))).thenReturn(localDir);
|
||||
@Test (timeout = 10000)
|
||||
public void testGetDataDirsFromURIs() throws Throwable {
|
||||
|
||||
DataNodeDiskChecker diskChecker = mock(DataNodeDiskChecker.class);
|
||||
doThrow(new IOException()).doThrow(new IOException()).doNothing()
|
||||
.when(diskChecker).checkDir(any(LocalFileSystem.class), any(Path.class));
|
||||
LocalFileSystem fs = mock(LocalFileSystem.class);
|
||||
Collection<URI> uris = Arrays.asList(new URI("file:/p1/"),
|
||||
new URI("file:/p2/"), new URI("file:/p3/"));
|
||||
|
||||
List<File> dirs = DataNode.getDataDirsFromURIs(uris, fs, normalPerm);
|
||||
|
||||
verify(fs, times(2)).setPermission(any(Path.class), eq(normalPerm));
|
||||
verify(fs, times(6)).getFileStatus(any(Path.class));
|
||||
assertEquals("number of valid data dirs", dirs.size(), 1);
|
||||
List<File> dirs = DataNode.getDataDirsFromURIs(uris, fs, diskChecker);
|
||||
assertEquals("number of valid data dirs", 1, dirs.size());
|
||||
String validDir = dirs.iterator().next().getPath();
|
||||
assertEquals("p3 should be valid", new File("/p3").getPath(), validDir);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -272,15 +272,15 @@ public abstract class FSImageTestUtil {
|
|||
for (File dir : dirs) {
|
||||
FSImageTransactionalStorageInspector inspector =
|
||||
inspectStorageDirectory(dir, NameNodeDirType.IMAGE);
|
||||
FSImageFile latestImage = inspector.getLatestImage();
|
||||
assertNotNull("No image in " + dir, latestImage);
|
||||
long thisTxId = latestImage.getCheckpointTxId();
|
||||
List<FSImageFile> latestImages = inspector.getLatestImages();
|
||||
assert(!latestImages.isEmpty());
|
||||
long thisTxId = latestImages.get(0).getCheckpointTxId();
|
||||
if (imageTxId != -1 && thisTxId != imageTxId) {
|
||||
fail("Storage directory " + dir + " does not have the same " +
|
||||
"last image index " + imageTxId + " as another");
|
||||
}
|
||||
imageTxId = thisTxId;
|
||||
imageFiles.add(inspector.getLatestImage().getFile());
|
||||
imageFiles.add(inspector.getLatestImages().get(0).getFile());
|
||||
}
|
||||
|
||||
assertFileContentsSame(imageFiles.toArray(new File[0]));
|
||||
|
@ -424,7 +424,7 @@ public abstract class FSImageTestUtil {
|
|||
new FSImageTransactionalStorageInspector();
|
||||
inspector.inspectDirectory(sd);
|
||||
|
||||
return inspector.getLatestImage().getFile();
|
||||
return inspector.getLatestImages().get(0).getFile();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -439,8 +439,8 @@ public abstract class FSImageTestUtil {
|
|||
new FSImageTransactionalStorageInspector();
|
||||
inspector.inspectDirectory(sd);
|
||||
|
||||
FSImageFile latestImage = inspector.getLatestImage();
|
||||
return (latestImage == null) ? null : latestImage.getFile();
|
||||
List<FSImageFile> latestImages = inspector.getLatestImages();
|
||||
return (latestImages.isEmpty()) ? null : latestImages.get(0).getFile();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -143,7 +143,7 @@ public class OfflineEditsViewerHelper {
|
|||
(DistributedFileSystem)cluster.getFileSystem();
|
||||
FileContext fc = FileContext.getFileContext(cluster.getURI(0), config);
|
||||
// OP_ADD 0, OP_SET_GENSTAMP 10
|
||||
Path pathFileCreate = new Path("/file_create");
|
||||
Path pathFileCreate = new Path("/file_create_u\1F431");
|
||||
FSDataOutputStream s = dfs.create(pathFileCreate);
|
||||
// OP_CLOSE 9
|
||||
s.close();
|
||||
|
|
|
@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
|
|||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.log4j.Level;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.log4j.PatternLayout;
|
||||
import org.apache.log4j.RollingFileAppender;
|
||||
|
@ -233,9 +234,15 @@ public class TestAuditLogs {
|
|||
|
||||
/** Sets up log4j logger for auditlogs */
|
||||
private void setupAuditLogs() throws IOException {
|
||||
// Shutdown the LogManager to release all logger open file handles.
|
||||
// Unfortunately, Apache commons logging library does not provide
|
||||
// means to release underlying loggers. For additional info look up
|
||||
// commons library FAQ.
|
||||
LogManager.shutdown();
|
||||
|
||||
File file = new File(auditLogFile);
|
||||
if (file.exists()) {
|
||||
file.delete();
|
||||
assertTrue(file.delete());
|
||||
}
|
||||
Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger();
|
||||
logger.setLevel(Level.INFO);
|
||||
|
|
|
@ -74,6 +74,8 @@ import org.apache.hadoop.net.NetUtils;
|
|||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
|
||||
import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
|
||||
import org.apache.hadoop.util.ExitUtil;
|
||||
import org.apache.hadoop.util.ExitUtil.ExitException;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.log4j.Level;
|
||||
import org.junit.After;
|
||||
|
@ -226,6 +228,111 @@ public class TestCheckpoint {
|
|||
toString().indexOf("storageDirToCheck") != -1);
|
||||
}
|
||||
|
||||
/*
|
||||
* Simulate exception during edit replay.
|
||||
*/
|
||||
@Test(timeout=30000)
|
||||
public void testReloadOnEditReplayFailure () throws IOException {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
FSDataOutputStream fos = null;
|
||||
SecondaryNameNode secondary = null;
|
||||
MiniDFSCluster cluster = null;
|
||||
FileSystem fs = null;
|
||||
|
||||
try {
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
|
||||
.build();
|
||||
cluster.waitActive();
|
||||
fs = cluster.getFileSystem();
|
||||
secondary = startSecondaryNameNode(conf);
|
||||
fos = fs.create(new Path("tmpfile0"));
|
||||
fos.write(new byte[] { 0, 1, 2, 3 });
|
||||
secondary.doCheckpoint();
|
||||
fos.write(new byte[] { 0, 1, 2, 3 });
|
||||
fos.hsync();
|
||||
|
||||
// Cause merge to fail in next checkpoint.
|
||||
Mockito.doThrow(new IOException(
|
||||
"Injecting failure during merge"))
|
||||
.when(faultInjector).duringMerge();
|
||||
|
||||
try {
|
||||
secondary.doCheckpoint();
|
||||
fail("Fault injection failed.");
|
||||
} catch (IOException ioe) {
|
||||
// This is expected.
|
||||
}
|
||||
Mockito.reset(faultInjector);
|
||||
|
||||
// The error must be recorded, so next checkpoint will reload image.
|
||||
fos.write(new byte[] { 0, 1, 2, 3 });
|
||||
fos.hsync();
|
||||
|
||||
assertTrue("Another checkpoint should have reloaded image",
|
||||
secondary.doCheckpoint());
|
||||
} finally {
|
||||
if (secondary != null) {
|
||||
secondary.shutdown();
|
||||
}
|
||||
if (fs != null) {
|
||||
fs.close();
|
||||
}
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
Mockito.reset(faultInjector);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Simulate 2NN exit due to too many merge failures.
|
||||
*/
|
||||
@Test(timeout=10000)
|
||||
public void testTooManyEditReplayFailures() throws IOException {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_MAX_RETRIES_KEY, "1");
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY, "1");
|
||||
|
||||
FSDataOutputStream fos = null;
|
||||
SecondaryNameNode secondary = null;
|
||||
MiniDFSCluster cluster = null;
|
||||
FileSystem fs = null;
|
||||
|
||||
try {
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
|
||||
.checkExitOnShutdown(false).build();
|
||||
cluster.waitActive();
|
||||
fs = cluster.getFileSystem();
|
||||
fos = fs.create(new Path("tmpfile0"));
|
||||
fos.write(new byte[] { 0, 1, 2, 3 });
|
||||
|
||||
// Cause merge to fail in next checkpoint.
|
||||
Mockito.doThrow(new IOException(
|
||||
"Injecting failure during merge"))
|
||||
.when(faultInjector).duringMerge();
|
||||
|
||||
secondary = startSecondaryNameNode(conf);
|
||||
secondary.doWork();
|
||||
// Fail if we get here.
|
||||
fail("2NN did not exit.");
|
||||
} catch (ExitException ee) {
|
||||
// ignore
|
||||
ExitUtil.resetFirstExitException();
|
||||
assertEquals("Max retries", 1, secondary.getMergeErrorCount() - 1);
|
||||
} finally {
|
||||
if (secondary != null) {
|
||||
secondary.shutdown();
|
||||
}
|
||||
if (fs != null) {
|
||||
fs.close();
|
||||
}
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
Mockito.reset(faultInjector);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Simulate namenode crashing after rolling edit log.
|
||||
*/
|
||||
|
@ -1304,6 +1411,60 @@ public class TestCheckpoint {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test NN restart if a failure happens in between creating the fsimage
|
||||
* MD5 file and renaming the fsimage.
|
||||
*/
|
||||
@Test(timeout=30000)
|
||||
public void testFailureBeforeRename () throws IOException {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
FSDataOutputStream fos = null;
|
||||
SecondaryNameNode secondary = null;
|
||||
MiniDFSCluster cluster = null;
|
||||
FileSystem fs = null;
|
||||
NameNode namenode = null;
|
||||
|
||||
try {
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
|
||||
.build();
|
||||
cluster.waitActive();
|
||||
namenode = cluster.getNameNode();
|
||||
fs = cluster.getFileSystem();
|
||||
secondary = startSecondaryNameNode(conf);
|
||||
fos = fs.create(new Path("tmpfile0"));
|
||||
fos.write(new byte[] { 0, 1, 2, 3 });
|
||||
secondary.doCheckpoint();
|
||||
fos.write(new byte[] { 0, 1, 2, 3 });
|
||||
fos.hsync();
|
||||
|
||||
// Cause merge to fail in next checkpoint.
|
||||
Mockito.doThrow(new IOException(
|
||||
"Injecting failure after MD5Rename"))
|
||||
.when(faultInjector).afterMD5Rename();
|
||||
|
||||
try {
|
||||
secondary.doCheckpoint();
|
||||
fail("Fault injection failed.");
|
||||
} catch (IOException ioe) {
|
||||
// This is expected.
|
||||
}
|
||||
Mockito.reset(faultInjector);
|
||||
// Namenode should still restart successfully
|
||||
cluster.restartNameNode();
|
||||
} finally {
|
||||
if (secondary != null) {
|
||||
secondary.shutdown();
|
||||
}
|
||||
if (fs != null) {
|
||||
fs.close();
|
||||
}
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
Mockito.reset(faultInjector);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test case where two secondary namenodes are checkpointing the same
|
||||
* NameNode. This differs from {@link #testMultipleSecondaryNamenodes()}
|
||||
|
|
|
@ -861,6 +861,11 @@ public class TestEditLog {
|
|||
public boolean isInProgress() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setMaxOpSize(int maxOpSize) {
|
||||
reader.setMaxOpSize(maxOpSize);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -57,7 +57,7 @@ public class TestFSImageStorageInspector {
|
|||
inspector.inspectDirectory(mockDir);
|
||||
assertEquals(2, inspector.foundImages.size());
|
||||
|
||||
FSImageFile latestImage = inspector.getLatestImage();
|
||||
FSImageFile latestImage = inspector.getLatestImages().get(0);
|
||||
assertEquals(456, latestImage.txId);
|
||||
assertSame(mockDir, latestImage.sd);
|
||||
assertTrue(inspector.isUpgradeFinalized());
|
||||
|
|
|
@ -120,12 +120,13 @@ public class TestHostsFiles {
|
|||
|
||||
InetSocketAddress nnHttpAddress = cluster.getNameNode().getHttpAddress();
|
||||
LOG.info("nnaddr = '" + nnHttpAddress + "'");
|
||||
URL nnjsp = new URL("http://" + nnHttpAddress.getHostName() + ":" + nnHttpAddress.getPort() + "/dfshealth.jsp");
|
||||
String nnHostName = nnHttpAddress.getHostName();
|
||||
URL nnjsp = new URL("http://" + nnHostName + ":" + nnHttpAddress.getPort() + "/dfshealth.jsp");
|
||||
LOG.info("fetching " + nnjsp);
|
||||
String dfshealthPage = StringEscapeUtils.unescapeHtml(DFSTestUtil.urlGet(nnjsp));
|
||||
LOG.info("got " + dfshealthPage);
|
||||
assertTrue("dfshealth should contain localhost, got:" + dfshealthPage,
|
||||
dfshealthPage.contains("localhost"));
|
||||
assertTrue("dfshealth should contain " + nnHostName + ", got:" + dfshealthPage,
|
||||
dfshealthPage.contains(nnHostName));
|
||||
|
||||
} finally {
|
||||
cluster.shutdown();
|
||||
|
|
|
@ -184,34 +184,41 @@ public class TestINodeFile {
|
|||
long fileLen = 1024;
|
||||
replication = 3;
|
||||
Configuration conf = new Configuration();
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
|
||||
replication).build();
|
||||
cluster.waitActive();
|
||||
FSNamesystem fsn = cluster.getNamesystem();
|
||||
FSDirectory fsdir = fsn.getFSDirectory();
|
||||
DistributedFileSystem dfs = cluster.getFileSystem();
|
||||
|
||||
// Create a file for test
|
||||
final Path dir = new Path("/dir");
|
||||
final Path file = new Path(dir, "file");
|
||||
DFSTestUtil.createFile(dfs, file, fileLen, replication, 0L);
|
||||
|
||||
// Check the full path name of the INode associating with the file
|
||||
INode fnode = fsdir.getINode(file.toString());
|
||||
assertEquals(file.toString(), fnode.getFullPathName());
|
||||
|
||||
// Call FSDirectory#unprotectedSetQuota which calls
|
||||
// INodeDirectory#replaceChild
|
||||
dfs.setQuota(dir, Long.MAX_VALUE - 1, replication * fileLen * 10);
|
||||
final Path newDir = new Path("/newdir");
|
||||
final Path newFile = new Path(newDir, "file");
|
||||
// Also rename dir
|
||||
dfs.rename(dir, newDir, Options.Rename.OVERWRITE);
|
||||
// /dir/file now should be renamed to /newdir/file
|
||||
fnode = fsdir.getINode(newFile.toString());
|
||||
// getFullPathName can return correct result only if the parent field of
|
||||
// child node is set correctly
|
||||
assertEquals(newFile.toString(), fnode.getFullPathName());
|
||||
MiniDFSCluster cluster = null;
|
||||
try {
|
||||
cluster =
|
||||
new MiniDFSCluster.Builder(conf).numDataNodes(replication).build();
|
||||
cluster.waitActive();
|
||||
FSNamesystem fsn = cluster.getNamesystem();
|
||||
FSDirectory fsdir = fsn.getFSDirectory();
|
||||
DistributedFileSystem dfs = cluster.getFileSystem();
|
||||
|
||||
// Create a file for test
|
||||
final Path dir = new Path("/dir");
|
||||
final Path file = new Path(dir, "file");
|
||||
DFSTestUtil.createFile(dfs, file, fileLen, replication, 0L);
|
||||
|
||||
// Check the full path name of the INode associating with the file
|
||||
INode fnode = fsdir.getINode(file.toString());
|
||||
assertEquals(file.toString(), fnode.getFullPathName());
|
||||
|
||||
// Call FSDirectory#unprotectedSetQuota which calls
|
||||
// INodeDirectory#replaceChild
|
||||
dfs.setQuota(dir, Long.MAX_VALUE - 1, replication * fileLen * 10);
|
||||
final Path newDir = new Path("/newdir");
|
||||
final Path newFile = new Path(newDir, "file");
|
||||
// Also rename dir
|
||||
dfs.rename(dir, newDir, Options.Rename.OVERWRITE);
|
||||
// /dir/file now should be renamed to /newdir/file
|
||||
fnode = fsdir.getINode(newFile.toString());
|
||||
// getFullPathName can return correct result only if the parent field of
|
||||
// child node is set correctly
|
||||
assertEquals(newFile.toString(), fnode.getFullPathName());
|
||||
} finally {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -385,41 +392,47 @@ public class TestINodeFile {
|
|||
Configuration conf = new Configuration();
|
||||
conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY,
|
||||
DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT);
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
|
||||
.build();
|
||||
cluster.waitActive();
|
||||
|
||||
FSNamesystem fsn = cluster.getNamesystem();
|
||||
long lastId = fsn.getLastInodeId();
|
||||
MiniDFSCluster cluster = null;
|
||||
try {
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
|
||||
assertTrue(lastId == 1001);
|
||||
FSNamesystem fsn = cluster.getNamesystem();
|
||||
long lastId = fsn.getLastInodeId();
|
||||
|
||||
// Create one directory and the last inode id should increase to 1002
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
Path path = new Path("/test1");
|
||||
assertTrue(fs.mkdirs(path));
|
||||
assertTrue(fsn.getLastInodeId() == 1002);
|
||||
assertTrue(lastId == 1001);
|
||||
|
||||
// Use namenode rpc to create a file
|
||||
NamenodeProtocols nnrpc = cluster.getNameNodeRpc();
|
||||
HdfsFileStatus fileStatus = nnrpc.create("/test1/file", new FsPermission(
|
||||
(short) 0755), "client",
|
||||
new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE)), true,
|
||||
(short) 1, 128 * 1024 * 1024L);
|
||||
assertTrue(fsn.getLastInodeId() == 1003);
|
||||
assertTrue(fileStatus.getFileId() == 1003);
|
||||
// Create one directory and the last inode id should increase to 1002
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
Path path = new Path("/test1");
|
||||
assertTrue(fs.mkdirs(path));
|
||||
assertTrue(fsn.getLastInodeId() == 1002);
|
||||
|
||||
// Rename doesn't increase inode id
|
||||
Path renamedPath = new Path("/test2");
|
||||
fs.rename(path, renamedPath);
|
||||
assertTrue(fsn.getLastInodeId() == 1003);
|
||||
// Use namenode rpc to create a file
|
||||
NamenodeProtocols nnrpc = cluster.getNameNodeRpc();
|
||||
HdfsFileStatus fileStatus = nnrpc.create("/test1/file", new FsPermission(
|
||||
(short) 0755), "client",
|
||||
new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE)), true,
|
||||
(short) 1, 128 * 1024 * 1024L);
|
||||
assertTrue(fsn.getLastInodeId() == 1003);
|
||||
assertTrue(fileStatus.getFileId() == 1003);
|
||||
|
||||
cluster.restartNameNode();
|
||||
cluster.waitActive();
|
||||
// Make sure empty editlog can be handled
|
||||
cluster.restartNameNode();
|
||||
cluster.waitActive();
|
||||
assertTrue(fsn.getLastInodeId() == 1003);
|
||||
// Rename doesn't increase inode id
|
||||
Path renamedPath = new Path("/test2");
|
||||
fs.rename(path, renamedPath);
|
||||
assertTrue(fsn.getLastInodeId() == 1003);
|
||||
|
||||
cluster.restartNameNode();
|
||||
cluster.waitActive();
|
||||
// Make sure empty editlog can be handled
|
||||
cluster.restartNameNode();
|
||||
cluster.waitActive();
|
||||
assertTrue(fsn.getLastInodeId() == 1003);
|
||||
} finally {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -83,6 +83,7 @@ public class TestNameNodeRecovery {
|
|||
elfos.close();
|
||||
elfos = null;
|
||||
elfis = new EditLogFileInputStream(TEST_LOG_NAME);
|
||||
elfis.setMaxOpSize(elts.getMaxOpSize());
|
||||
|
||||
// reading through normally will get you an exception
|
||||
Set<Long> validTxIds = elts.getValidTxIds();
|
||||
|
@ -143,7 +144,7 @@ public class TestNameNodeRecovery {
|
|||
/**
|
||||
* A test scenario for the edit log
|
||||
*/
|
||||
private interface EditLogTestSetup {
|
||||
private static abstract class EditLogTestSetup {
|
||||
/**
|
||||
* Set up the edit log.
|
||||
*/
|
||||
|
@ -162,6 +163,13 @@ public class TestNameNodeRecovery {
|
|||
* edit log.
|
||||
**/
|
||||
abstract public Set<Long> getValidTxIds();
|
||||
|
||||
/**
|
||||
* Return the maximum opcode size we will use for input.
|
||||
*/
|
||||
public int getMaxOpSize() {
|
||||
return DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_DEFAULT;
|
||||
}
|
||||
}
|
||||
|
||||
static void padEditLog(EditLogOutputStream elos, int paddingLength)
|
||||
|
@ -182,10 +190,10 @@ public class TestNameNodeRecovery {
|
|||
}
|
||||
|
||||
static void addDeleteOpcode(EditLogOutputStream elos,
|
||||
OpInstanceCache cache) throws IOException {
|
||||
OpInstanceCache cache, long txId, String path) throws IOException {
|
||||
DeleteOp op = DeleteOp.getInstance(cache);
|
||||
op.setTransactionId(0x0);
|
||||
op.setPath("/foo");
|
||||
op.setTransactionId(txId);
|
||||
op.setPath(path);
|
||||
op.setTimestamp(0);
|
||||
elos.write(op);
|
||||
}
|
||||
|
@ -198,7 +206,7 @@ public class TestNameNodeRecovery {
|
|||
* able to handle any amount of padding (including no padding) without
|
||||
* throwing an exception.
|
||||
*/
|
||||
private static class EltsTestEmptyLog implements EditLogTestSetup {
|
||||
private static class EltsTestEmptyLog extends EditLogTestSetup {
|
||||
private int paddingLength;
|
||||
|
||||
public EltsTestEmptyLog(int paddingLength) {
|
||||
|
@ -242,6 +250,42 @@ public class TestNameNodeRecovery {
|
|||
3 * EditLogFileOutputStream.MIN_PREALLOCATION_LENGTH));
|
||||
}
|
||||
|
||||
/**
|
||||
* Test using a non-default maximum opcode length.
|
||||
*/
|
||||
private static class EltsTestNonDefaultMaxOpSize extends EditLogTestSetup {
|
||||
public EltsTestNonDefaultMaxOpSize() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addTransactionsToLog(EditLogOutputStream elos,
|
||||
OpInstanceCache cache) throws IOException {
|
||||
addDeleteOpcode(elos, cache, 0, "/foo");
|
||||
addDeleteOpcode(elos, cache, 1,
|
||||
"/supercalifragalisticexpialadocius.supercalifragalisticexpialadocius");
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLastValidTxId() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<Long> getValidTxIds() {
|
||||
return Sets.newHashSet(0L);
|
||||
}
|
||||
|
||||
public int getMaxOpSize() {
|
||||
return 30;
|
||||
}
|
||||
}
|
||||
|
||||
/** Test an empty edit log with extra-long padding */
|
||||
@Test(timeout=180000)
|
||||
public void testNonDefaultMaxOpSize() throws IOException {
|
||||
runEditLogTest(new EltsTestNonDefaultMaxOpSize());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test the scenario where an edit log contains some padding (0xff) bytes
|
||||
* followed by valid opcode data.
|
||||
|
@ -249,7 +293,7 @@ public class TestNameNodeRecovery {
|
|||
* These edit logs are corrupt, but all the opcodes should be recoverable
|
||||
* with recovery mode.
|
||||
*/
|
||||
private static class EltsTestOpcodesAfterPadding implements EditLogTestSetup {
|
||||
private static class EltsTestOpcodesAfterPadding extends EditLogTestSetup {
|
||||
private int paddingLength;
|
||||
|
||||
public EltsTestOpcodesAfterPadding(int paddingLength) {
|
||||
|
@ -260,7 +304,7 @@ public class TestNameNodeRecovery {
|
|||
public void addTransactionsToLog(EditLogOutputStream elos,
|
||||
OpInstanceCache cache) throws IOException {
|
||||
padEditLog(elos, paddingLength);
|
||||
addDeleteOpcode(elos, cache);
|
||||
addDeleteOpcode(elos, cache, 0, "/foo");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -286,7 +330,7 @@ public class TestNameNodeRecovery {
|
|||
3 * EditLogFileOutputStream.MIN_PREALLOCATION_LENGTH));
|
||||
}
|
||||
|
||||
private static class EltsTestGarbageInEditLog implements EditLogTestSetup {
|
||||
private static class EltsTestGarbageInEditLog extends EditLogTestSetup {
|
||||
final private long BAD_TXID = 4;
|
||||
final private long MAX_TXID = 10;
|
||||
|
||||
|
|
|
@ -158,7 +158,7 @@ public class TestProcessCorruptBlocks {
|
|||
* (corrupt replica should be removed since number of good
|
||||
* replicas (1) is equal to replication factor (1))
|
||||
*/
|
||||
@Test
|
||||
@Test(timeout=20000)
|
||||
public void testWithReplicationFactorAsOne() throws Exception {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L);
|
||||
|
@ -183,9 +183,14 @@ public class TestProcessCorruptBlocks {
|
|||
namesystem.setReplication(fileName.toString(), (short) 1);
|
||||
|
||||
// wait for 3 seconds so that all block reports are processed.
|
||||
try {
|
||||
Thread.sleep(3000);
|
||||
} catch (InterruptedException ignored) {
|
||||
for (int i = 0; i < 10; i++) {
|
||||
try {
|
||||
Thread.sleep(1000);
|
||||
} catch (InterruptedException ignored) {
|
||||
}
|
||||
if (countReplicas(namesystem, block).corruptReplicas() == 0) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
assertEquals(1, countReplicas(namesystem, block).liveReplicas());
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue