Merge r1426019 through r1432788 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1432796 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2013-01-14 03:44:35 +00:00
commit 90d6d55a4b
321 changed files with 11127 additions and 2509 deletions

View File

@ -148,6 +148,8 @@ Trunk (Unreleased)
BUG FIXES
HADOOP-8419. Fixed GzipCode NPE reset for IBM JDK. (Yu Li via eyang)
HADOOP-9041. FsUrlStreamHandlerFactory could cause an infinite loop in
FileSystem initialization. (Yanbo Liang and Radim Kolar via llu)
@ -309,6 +311,8 @@ Trunk (Unreleased)
HADOOP-8957 AbstractFileSystem#IsValidName should be overridden for
embedded file systems like ViewFs (Chris Nauroth via Sanjay Radia)
HADOOP-9139 improve killKdc.sh (Ivan A. Veselovsky via bobby)
OPTIMIZATIONS
HADOOP-7761. Improve the performance of raw comparisons. (todd)
@ -420,6 +424,15 @@ Release 2.0.3-alpha - Unreleased
HADOOP-8427. Convert Forrest docs to APT, incremental. (adi2 via tucu)
HADOOP-9173. Add security token protobuf definition to common and
use it in hdfs. (suresh)
HADOOP-9119. Add test to FileSystemContractBaseTest to verify integrity
of overwritten files. (Steve Loughran via suresh)
HADOOP-9192. Move token related request/response messages to common.
(suresh)
OPTIMIZATIONS
HADOOP-8866. SampleQuantiles#query is O(N^2) instead of O(N). (Andrew Wang
@ -516,6 +529,14 @@ Release 2.0.3-alpha - Unreleased
HADOOP-9153. Support createNonRecursive in ViewFileSystem.
(Sandy Ryza via tomwhite)
HADOOP-9181. Set daemon flag for HttpServer's QueuedThreadPool.
(Liang Xie via suresh)
HADOOP-9155. FsPermission should have different default value, 777 for
directory and 666 for file. (Binglin Chang via atm)
HADOOP-9183. Potential deadlock in ActiveStandbyElector. (tomwhite)
Release 2.0.2-alpha - 2012-09-07
INCOMPATIBLE CHANGES

View File

@ -282,6 +282,10 @@
<!-- protobuf generated code -->
<Class name="~org\.apache\.hadoop\.ha\.proto\.ZKFCProtocolProtos.*"/>
</Match>
<Match>
<!-- protobuf generated code -->
<Class name="~org\.apache\.hadoop\.security\.proto\.SecurityProtos.*"/>
</Match>
<!--
Manually checked, misses child thread manually syncing on parent's intrinsic lock.

View File

@ -381,6 +381,7 @@
<argument>src/main/proto/RpcHeader.proto</argument>
<argument>src/main/proto/ZKFCProtocol.proto</argument>
<argument>src/main/proto/ProtobufRpcEngine.proto</argument>
<argument>src/main/proto/Security.proto</argument>
</arguments>
</configuration>
</execution>

View File

@ -172,7 +172,25 @@ import org.apache.hadoop.util.ShutdownHookManager;
public final class FileContext {
public static final Log LOG = LogFactory.getLog(FileContext.class);
/**
* Default permission for directory and symlink
* In previous versions, this default permission was also used to
* create files, so files created end up with ugo+x permission.
* See HADOOP-9155 for detail.
* Two new constants are added to solve this, please use
* {@link FileContext#DIR_DEFAULT_PERM} for directory, and use
* {@link FileContext#FILE_DEFAULT_PERM} for file.
* This constant is kept for compatibility.
*/
public static final FsPermission DEFAULT_PERM = FsPermission.getDefault();
/**
* Default permission for directory
*/
public static final FsPermission DIR_DEFAULT_PERM = FsPermission.getDirDefault();
/**
* Default permission for file
*/
public static final FsPermission FILE_DEFAULT_PERM = FsPermission.getFileDefault();
/**
* Priority of the FileContext shutdown hook.
@ -656,7 +674,7 @@ public final class FileContext {
CreateOpts.Perms permOpt =
(CreateOpts.Perms) CreateOpts.getOpt(CreateOpts.Perms.class, opts);
FsPermission permission = (permOpt != null) ? permOpt.getValue() :
FsPermission.getDefault();
FILE_DEFAULT_PERM;
permission = permission.applyUMask(umask);
final CreateOpts[] updatedOpts =
@ -704,7 +722,7 @@ public final class FileContext {
IOException {
final Path absDir = fixRelativePart(dir);
final FsPermission absFerms = (permission == null ?
FsPermission.getDefault() : permission).applyUMask(umask);
FsPermission.getDirDefault() : permission).applyUMask(umask);
new FSLinkResolver<Void>() {
@Override
public Void next(final AbstractFileSystem fs, final Path p)
@ -2157,7 +2175,7 @@ public final class FileContext {
FileStatus fs = FileContext.this.getFileStatus(qSrc);
if (fs.isDirectory()) {
checkDependencies(qSrc, qDst);
mkdir(qDst, FsPermission.getDefault(), true);
mkdir(qDst, FsPermission.getDirDefault(), true);
FileStatus[] contents = listStatus(qSrc);
for (FileStatus content : contents) {
copy(makeQualified(content.getPath()), makeQualified(new Path(qDst,

View File

@ -79,8 +79,15 @@ public class FileStatus implements Writable, Comparable {
this.blocksize = blocksize;
this.modification_time = modification_time;
this.access_time = access_time;
this.permission = (permission == null) ?
FsPermission.getDefault() : permission;
if (permission != null) {
this.permission = permission;
} else if (isdir) {
this.permission = FsPermission.getDirDefault();
} else if (symlink!=null) {
this.permission = FsPermission.getDefault();
} else {
this.permission = FsPermission.getFileDefault();
}
this.owner = (owner == null) ? "" : owner;
this.group = (group == null) ? "" : group;
this.symlink = symlink;
@ -217,7 +224,7 @@ public class FileStatus implements Writable, Comparable {
*/
protected void setPermission(FsPermission permission) {
this.permission = (permission == null) ?
FsPermission.getDefault() : permission;
FsPermission.getFileDefault() : permission;
}
/**

View File

@ -850,7 +850,7 @@ public abstract class FileSystem extends Configured implements Closeable {
long blockSize,
Progressable progress
) throws IOException {
return this.create(f, FsPermission.getDefault().applyUMask(
return this.create(f, FsPermission.getFileDefault().applyUMask(
FsPermission.getUMask(getConf())), overwrite, bufferSize,
replication, blockSize, progress);
}
@ -1030,7 +1030,7 @@ public abstract class FileSystem extends Configured implements Closeable {
boolean overwrite,
int bufferSize, short replication, long blockSize,
Progressable progress) throws IOException {
return this.createNonRecursive(f, FsPermission.getDefault(),
return this.createNonRecursive(f, FsPermission.getFileDefault(),
overwrite, bufferSize, replication, blockSize, progress);
}
@ -1866,7 +1866,7 @@ public abstract class FileSystem extends Configured implements Closeable {
* Call {@link #mkdirs(Path, FsPermission)} with default permission.
*/
public boolean mkdirs(Path f) throws IOException {
return mkdirs(f, FsPermission.getDefault());
return mkdirs(f, FsPermission.getDirDefault());
}
/**

View File

@ -224,7 +224,7 @@ public class FTPFileSystem extends FileSystem {
}
Path parent = absolute.getParent();
if (parent == null || !mkdirs(client, parent, FsPermission.getDefault())) {
if (parent == null || !mkdirs(client, parent, FsPermission.getDirDefault())) {
parent = (parent == null) ? new Path("/") : parent;
disconnect(client);
throw new IOException("create(): Mkdirs failed to create: " + parent);
@ -484,7 +484,7 @@ public class FTPFileSystem extends FileSystem {
if (!exists(client, absolute)) {
Path parent = absolute.getParent();
created = (parent == null || mkdirs(client, parent, FsPermission
.getDefault()));
.getDirDefault()));
if (created) {
String parentDir = parent.toUri().getPath();
client.changeWorkingDirectory(parentDir);

View File

@ -85,7 +85,7 @@ public class RawLocalFs extends DelegateToFileSystem {
"system: "+target.toString());
}
if (createParent) {
mkdir(link.getParent(), FsPermission.getDefault(), true);
mkdir(link.getParent(), FsPermission.getDirDefault(), true);
}
// NB: Use createSymbolicLink in java.nio.file.Path once available
try {

View File

@ -275,11 +275,34 @@ public class FsPermission implements Writable {
conf.setInt(DEPRECATED_UMASK_LABEL, umask.toShort());
}
/** Get the default permission. */
/**
* Get the default permission for directory and symlink.
* In previous versions, this default permission was also used to
* create files, so files created end up with ugo+x permission.
* See HADOOP-9155 for detail.
* Two new methods are added to solve this, please use
* {@link FsPermission#getDirDefault()} for directory, and use
* {@link FsPermission#getFileDefault()} for file.
* This method is kept for compatibility.
*/
public static FsPermission getDefault() {
return new FsPermission((short)00777);
}
/**
* Get the default permission for directory.
*/
public static FsPermission getDirDefault() {
return new FsPermission((short)00777);
}
/**
* Get the default permission for file.
*/
public static FsPermission getFileDefault() {
return new FsPermission((short)00666);
}
/**
* Create a FsPermission from a Unix symbolic permission string
* @param unixSymbolicPermission e.g. "-rw-rw-rw-"

View File

@ -613,7 +613,7 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
// Unfortunately, the ZooKeeper constructor connects to ZooKeeper and
// may trigger the Connected event immediately. So, if we register the
// watcher after constructing ZooKeeper, we may miss that event. Instead,
// we construct the watcher first, and have it queue any events it receives
// we construct the watcher first, and have it block any events it receives
// before we can set its ZooKeeper reference.
WatcherWithClientRef watcher = new WatcherWithClientRef();
ZooKeeper zk = new ZooKeeper(zkHostPort, zkSessionTimeout, watcher);
@ -1002,19 +1002,17 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
private CountDownLatch hasReceivedEvent = new CountDownLatch(1);
/**
* If any events arrive before the reference to ZooKeeper is set,
* they get queued up and later forwarded when the reference is
* available.
* Latch used to wait until the reference to ZooKeeper is set.
*/
private final List<WatchedEvent> queuedEvents = Lists.newLinkedList();
private CountDownLatch hasSetZooKeeper = new CountDownLatch(1);
private WatcherWithClientRef() {
}
private WatcherWithClientRef(ZooKeeper zk) {
this.zk = zk;
setZooKeeperRef(zk);
}
/**
* Waits for the next event from ZooKeeper to arrive.
*
@ -1029,9 +1027,7 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
if (!hasReceivedEvent.await(connectionTimeoutMs, TimeUnit.MILLISECONDS)) {
LOG.error("Connection timed out: couldn't connect to ZooKeeper in "
+ connectionTimeoutMs + " milliseconds");
synchronized (this) {
zk.close();
}
zk.close();
throw KeeperException.create(Code.CONNECTIONLOSS);
}
} catch (InterruptedException e) {
@ -1041,29 +1037,18 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
}
}
private synchronized void setZooKeeperRef(ZooKeeper zk) {
private void setZooKeeperRef(ZooKeeper zk) {
Preconditions.checkState(this.zk == null,
"zk already set -- must be set exactly once");
this.zk = zk;
for (WatchedEvent e : queuedEvents) {
forwardEvent(e);
}
queuedEvents.clear();
hasSetZooKeeper.countDown();
}
@Override
public synchronized void process(WatchedEvent event) {
if (zk != null) {
forwardEvent(event);
} else {
queuedEvents.add(event);
}
}
private void forwardEvent(WatchedEvent event) {
public void process(WatchedEvent event) {
hasReceivedEvent.countDown();
try {
hasSetZooKeeper.await(zkSessionTimeout, TimeUnit.MILLISECONDS);
ActiveStandbyElector.this.processWatchEvent(
zk, event);
} catch (Throwable t) {

View File

@ -245,6 +245,7 @@ public class HttpServer implements FilterContainer {
// default value (currently 250).
QueuedThreadPool threadPool = maxThreads == -1 ?
new QueuedThreadPool() : new QueuedThreadPool(maxThreads);
threadPool.setDaemon(true);
webServer.setThreadPool(threadPool);
final String appDir = getWebAppsPath(name);

View File

@ -40,14 +40,74 @@ public class GzipCodec extends DefaultCodec {
protected static class GzipOutputStream extends CompressorStream {
private static class ResetableGZIPOutputStream extends GZIPOutputStream {
private static final int TRAILER_SIZE = 8;
public static final String JVMVendor= System.getProperty("java.vendor");
public static final String JVMVersion= System.getProperty("java.version");
private static final boolean HAS_BROKEN_FINISH =
(JVMVendor.contains("IBM") && JVMVersion.contains("1.6.0"));
public ResetableGZIPOutputStream(OutputStream out) throws IOException {
super(out);
}
public void resetState() throws IOException {
def.reset();
}
/**
* Override this method for HADOOP-8419.
* Override because IBM implementation calls def.end() which
* causes problem when reseting the stream for reuse.
*
*/
@Override
public void finish() throws IOException {
if (HAS_BROKEN_FINISH) {
if (!def.finished()) {
def.finish();
while (!def.finished()) {
int i = def.deflate(this.buf, 0, this.buf.length);
if ((def.finished()) && (i <= this.buf.length - TRAILER_SIZE)) {
writeTrailer(this.buf, i);
i += TRAILER_SIZE;
out.write(this.buf, 0, i);
return;
}
if (i > 0) {
out.write(this.buf, 0, i);
}
}
byte[] arrayOfByte = new byte[TRAILER_SIZE];
writeTrailer(arrayOfByte, 0);
out.write(arrayOfByte);
}
} else {
super.finish();
}
}
/** re-implement for HADOOP-8419 because the relative method in jdk is invisible */
private void writeTrailer(byte[] paramArrayOfByte, int paramInt)
throws IOException {
writeInt((int)this.crc.getValue(), paramArrayOfByte, paramInt);
writeInt(this.def.getTotalIn(), paramArrayOfByte, paramInt + 4);
}
/** re-implement for HADOOP-8419 because the relative method in jdk is invisible */
private void writeInt(int paramInt1, byte[] paramArrayOfByte, int paramInt2)
throws IOException {
writeShort(paramInt1 & 0xFFFF, paramArrayOfByte, paramInt2);
writeShort(paramInt1 >> 16 & 0xFFFF, paramArrayOfByte, paramInt2 + 2);
}
/** re-implement for HADOOP-8419 because the relative method in jdk is invisible */
private void writeShort(int paramInt1, byte[] paramArrayOfByte, int paramInt2)
throws IOException {
paramArrayOfByte[paramInt2] = (byte)(paramInt1 & 0xFF);
paramArrayOfByte[(paramInt2 + 1)] = (byte)(paramInt1 >> 8 & 0xFF);
}
}
public GzipOutputStream(OutputStream out) throws IOException {

View File

@ -0,0 +1,31 @@
/**
* 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.util;
import org.apache.hadoop.classification.InterfaceAudience;
/**
* Generic ID generator
* used for generating various types of number sequences.
*/
@InterfaceAudience.Private
public interface IdGenerator {
/** Increment and then return the next value. */
public long nextValue();
}

View File

@ -0,0 +1,83 @@
/**
* 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.util;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.classification.InterfaceAudience;
/**
* Sequential number generator.
*
* This class is thread safe.
*/
@InterfaceAudience.Private
public abstract class SequentialNumber implements IdGenerator {
private final AtomicLong currentValue;
/** Create a new instance with the given initial value. */
protected SequentialNumber(final long initialValue) {
currentValue = new AtomicLong(initialValue);
}
/** @return the current value. */
public long getCurrentValue() {
return currentValue.get();
}
/** Set current value. */
public void setCurrentValue(long value) {
currentValue.set(value);
}
/** Increment and then return the next value. */
public long nextValue() {
return currentValue.incrementAndGet();
}
/** Skip to the new value. */
public void skipTo(long newValue) throws IllegalStateException {
for(;;) {
final long c = getCurrentValue();
if (newValue < c) {
throw new IllegalStateException(
"Cannot skip to less than the current value (="
+ c + "), where newValue=" + newValue);
}
if (currentValue.compareAndSet(c, newValue)) {
return;
}
}
}
@Override
public boolean equals(final Object that) {
if (that == null || this.getClass() != that.getClass()) {
return false;
}
final AtomicLong thatValue = ((SequentialNumber)that).currentValue;
return currentValue.equals(thatValue);
}
@Override
public int hashCode() {
final long v = currentValue.get();
return (int)v ^ (int)(v >>> 32);
}
}

View File

@ -0,0 +1,57 @@
/**
* 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.
*/
option java_package = "org.apache.hadoop.security.proto";
option java_outer_classname = "SecurityProtos";
option java_generic_services = true;
option java_generate_equals_and_hash = true;
package hadoop.common;
/**
* Security token identifier
*/
message TokenProto {
required bytes identifier = 1;
required bytes password = 2;
required string kind = 3;
required string service = 4;
}
message GetDelegationTokenRequestProto {
required string renewer = 1;
}
message GetDelegationTokenResponseProto {
optional hadoop.common.TokenProto token = 1;
}
message RenewDelegationTokenRequestProto {
required hadoop.common.TokenProto token = 1;
}
message RenewDelegationTokenResponseProto {
required uint64 newExpiryTime = 1;
}
message CancelDelegationTokenRequestProto {
required hadoop.common.TokenProto token = 1;
}
message CancelDelegationTokenResponseProto { // void response
}

View File

@ -95,7 +95,7 @@ public abstract class FileContextPermissionBase {
String filename = "foo";
Path f = getTestRootPath(fc, filename);
createFile(fc, filename);
doFilePermissionCheck(FileContext.DEFAULT_PERM.applyUMask(fc.getUMask()),
doFilePermissionCheck(FileContext.FILE_DEFAULT_PERM.applyUMask(fc.getUMask()),
fc.getFileStatus(f).getPermission());
}

View File

@ -23,12 +23,9 @@ import java.io.IOException;
import junit.framework.TestCase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
/**
@ -45,15 +42,13 @@ import org.apache.hadoop.fs.permission.FsPermission;
* </p>
*/
public abstract class FileSystemContractBaseTest extends TestCase {
private static final Log LOG =
LogFactory.getLog(FileSystemContractBaseTest.class);
protected final static String TEST_UMASK = "062";
protected FileSystem fs;
protected byte[] data = new byte[getBlockSize() * 2]; // two blocks of data
{
for (int i = 0; i < data.length; i++) {
data[i] = (byte) (i % 10);
}
}
protected byte[] data = dataset(getBlockSize() * 2, 0, 255);
@Override
protected void tearDown() throws Exception {
fs.delete(path("/test"), true);
@ -235,35 +230,16 @@ public abstract class FileSystemContractBaseTest extends TestCase {
public void testWriteReadAndDeleteTwoBlocks() throws Exception {
writeReadAndDelete(getBlockSize() * 2);
}
/**
* Write a dataset, read it back in and verify that they match.
* Afterwards, the file is deleted.
* @param len length of data
* @throws IOException on IO failures
*/
protected void writeReadAndDelete(int len) throws IOException {
Path path = path("/test/hadoop/file");
fs.mkdirs(path.getParent());
FSDataOutputStream out = fs.create(path, false,
fs.getConf().getInt("io.file.buffer.size", 4096),
(short) 1, getBlockSize());
out.write(data, 0, len);
out.close();
assertTrue("Exists", fs.exists(path));
assertEquals("Length", len, fs.getFileStatus(path).getLen());
FSDataInputStream in = fs.open(path);
byte[] buf = new byte[len];
in.readFully(0, buf);
in.close();
assertEquals(len, buf.length);
for (int i = 0; i < buf.length; i++) {
assertEquals("Position " + i, data[i], buf[i]);
}
assertTrue("Deleted", fs.delete(path, false));
assertFalse("No longer exists", fs.exists(path));
writeAndRead(path, data, len, false, true);
}
public void testOverwrite() throws IOException {
@ -494,4 +470,147 @@ public abstract class FileSystemContractBaseTest extends TestCase {
assertEquals("Source exists", srcExists, fs.exists(src));
assertEquals("Destination exists", dstExists, fs.exists(dst));
}
/**
* Verify that if you take an existing file and overwrite it, the new values
* get picked up.
* This is a test for the behavior of eventually consistent
* filesystems.
*
* @throws Exception on any failure
*/
public void testOverWriteAndRead() throws Exception {
int blockSize = getBlockSize();
byte[] filedata1 = dataset(blockSize * 2, 'A', 26);
byte[] filedata2 = dataset(blockSize * 2, 'a', 26);
Path path = path("/test/hadoop/file-overwrite");
writeAndRead(path, filedata1, blockSize, true, false);
writeAndRead(path, filedata2, blockSize, true, false);
writeAndRead(path, filedata1, blockSize * 2, true, false);
writeAndRead(path, filedata2, blockSize * 2, true, false);
writeAndRead(path, filedata1, blockSize, true, false);
writeAndRead(path, filedata2, blockSize * 2, true, false);
}
/**
*
* Write a file and read it in, validating the result. Optional flags control
* whether file overwrite operations should be enabled, and whether the
* file should be deleted afterwards.
*
* If there is a mismatch between what was written and what was expected,
* a small range of bytes either side of the first error are logged to aid
* diagnosing what problem occurred -whether it was a previous file
* or a corrupting of the current file. This assumes that two
* sequential runs to the same path use datasets with different character
* moduli.
*
* @param path path to write to
* @param len length of data
* @param overwrite should the create option allow overwrites?
* @param delete should the file be deleted afterwards? -with a verification
* that it worked. Deletion is not attempted if an assertion has failed
* earlier -it is not in a <code>finally{}</code> block.
* @throws IOException IO problems
*/
protected void writeAndRead(Path path, byte[] src, int len,
boolean overwrite,
boolean delete) throws IOException {
assertTrue("Not enough data in source array to write " + len + " bytes",
src.length >= len);
fs.mkdirs(path.getParent());
FSDataOutputStream out = fs.create(path, overwrite,
fs.getConf().getInt("io.file.buffer.size",
4096),
(short) 1, getBlockSize());
out.write(src, 0, len);
out.close();
assertTrue("Exists", fs.exists(path));
assertEquals("Length", len, fs.getFileStatus(path).getLen());
FSDataInputStream in = fs.open(path);
byte[] buf = new byte[len];
in.readFully(0, buf);
in.close();
assertEquals(len, buf.length);
int errors = 0;
int first_error_byte = -1;
for (int i = 0; i < len; i++) {
if (src[i] != buf[i]) {
if (errors == 0) {
first_error_byte = i;
}
errors++;
}
}
if (errors > 0) {
String message = String.format(" %d errors in file of length %d",
errors, len);
LOG.warn(message);
// the range either side of the first error to print
// this is a purely arbitrary number, to aid user debugging
final int overlap = 10;
for (int i = Math.max(0, first_error_byte - overlap);
i < Math.min(first_error_byte + overlap, len);
i++) {
byte actual = buf[i];
byte expected = src[i];
String letter = toChar(actual);
String line = String.format("[%04d] %2x %s\n", i, actual, letter);
if (expected != actual) {
line = String.format("[%04d] %2x %s -expected %2x %s\n",
i,
actual,
letter,
expected,
toChar(expected));
}
LOG.warn(line);
}
fail(message);
}
if (delete) {
boolean deleted = fs.delete(path, false);
assertTrue("Deleted", deleted);
assertFalse("No longer exists", fs.exists(path));
}
}
/**
* Convert a byte to a character for printing. If the
* byte value is < 32 -and hence unprintable- the byte is
* returned as a two digit hex value
* @param b byte
* @return the printable character string
*/
protected String toChar(byte b) {
if (b >= 0x20) {
return Character.toString((char) b);
} else {
return String.format("%02x", b);
}
}
/**
* Create a dataset for use in the tests; all data is in the range
* base to (base+modulo-1) inclusive
* @param len length of data
* @param base base of the data
* @param modulo the modulo
* @return the newly generated dataset
*/
protected byte[] dataset(int len, int base, int modulo) {
byte[] dataset = new byte[len];
for (int i = 0; i < len; i++) {
dataset[i] = (byte) (base + (i % modulo));
}
return dataset;
}
}

View File

@ -121,7 +121,7 @@ public class TestFileStatus {
FileStatus fileStatus = new FileStatus(LENGTH, isdir,
REPLICATION, BLKSIZE, MTIME, PATH);
validateAccessors(fileStatus, LENGTH, isdir, REPLICATION, BLKSIZE, MTIME,
0, FsPermission.getDefault(), "", "", null, PATH);
0, FsPermission.getDirDefault(), "", "", null, PATH);
}
/**
@ -131,7 +131,7 @@ public class TestFileStatus {
public void constructorBlank() throws IOException {
FileStatus fileStatus = new FileStatus();
validateAccessors(fileStatus, 0, false, 0, 0, 0,
0, FsPermission.getDefault(), "", "", null, null);
0, FsPermission.getFileDefault(), "", "", null, null);
}
/**

View File

@ -24,6 +24,8 @@ import org.apache.hadoop.conf.Configuration;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.apache.hadoop.fs.FileContextTestHelper;
import org.apache.hadoop.fs.permission.FsPermission;
public class TestLocalFSFileContextMainOperations extends FileContextMainOperationsBaseTest {
@ -47,4 +49,14 @@ public class TestLocalFSFileContextMainOperations extends FileContextMainOperati
FileContext fc1 = FileContext.getLocalFSFileContext();
Assert.assertTrue(fc1 != fc);
}
@Test
public void testDefaultFilePermission() throws IOException {
Path file = FileContextTestHelper.getTestRootPath(fc,
"testDefaultFilePermission");
FileContextTestHelper.createFile(fc, file);
FsPermission expect = FileContext.FILE_DEFAULT_PERM.applyUMask(fc.getUMask());
Assert.assertEquals(expect, fc.getFileStatus(file)
.getPermission());
}
}

View File

@ -73,7 +73,7 @@ public class TestLocalFileSystemPermission extends TestCase {
try {
FsPermission initialPermission = getPermission(localfs, f);
System.out.println(filename + ": " + initialPermission);
assertEquals(FsPermission.getDefault().applyUMask(FsPermission.getUMask(conf)), initialPermission);
assertEquals(FsPermission.getFileDefault().applyUMask(FsPermission.getUMask(conf)), initialPermission);
}
catch(Exception e) {
System.out.println(StringUtils.stringifyException(e));

View File

@ -0,0 +1,161 @@
/**
* 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.io.compress;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.RandomDatum;
import org.apache.hadoop.io.compress.zlib.ZlibFactory;
import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionLevel;
import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionStrategy;
import org.apache.hadoop.util.ReflectionUtils;
import junit.framework.TestCase;
public class TestCompressionStreamReuse extends TestCase {
private static final Log LOG = LogFactory
.getLog(TestCompressionStreamReuse.class);
private Configuration conf = new Configuration();
private int count = 10000;
private int seed = new Random().nextInt();
public void testBZip2Codec() throws IOException {
resetStateTest(conf, seed, count,
"org.apache.hadoop.io.compress.BZip2Codec");
}
public void testGzipCompressStreamReuse() throws IOException {
resetStateTest(conf, seed, count,
"org.apache.hadoop.io.compress.GzipCodec");
}
public void testGzipCompressStreamReuseWithParam() throws IOException {
Configuration conf = new Configuration(this.conf);
ZlibFactory
.setCompressionLevel(conf, CompressionLevel.BEST_COMPRESSION);
ZlibFactory.setCompressionStrategy(conf,
CompressionStrategy.HUFFMAN_ONLY);
resetStateTest(conf, seed, count,
"org.apache.hadoop.io.compress.GzipCodec");
}
private static void resetStateTest(Configuration conf, int seed, int count,
String codecClass) throws IOException {
// Create the codec
CompressionCodec codec = null;
try {
codec = (CompressionCodec) ReflectionUtils.newInstance(conf
.getClassByName(codecClass), conf);
} catch (ClassNotFoundException cnfe) {
throw new IOException("Illegal codec!");
}
LOG.info("Created a Codec object of type: " + codecClass);
// Generate data
DataOutputBuffer data = new DataOutputBuffer();
RandomDatum.Generator generator = new RandomDatum.Generator(seed);
for (int i = 0; i < count; ++i) {
generator.next();
RandomDatum key = generator.getKey();
RandomDatum value = generator.getValue();
key.write(data);
value.write(data);
}
LOG.info("Generated " + count + " records");
// Compress data
DataOutputBuffer compressedDataBuffer = new DataOutputBuffer();
DataOutputStream deflateOut = new DataOutputStream(
new BufferedOutputStream(compressedDataBuffer));
CompressionOutputStream deflateFilter = codec
.createOutputStream(deflateOut);
deflateFilter.write(data.getData(), 0, data.getLength());
deflateFilter.finish();
deflateFilter.flush();
LOG.info("Finished compressing data");
// reset deflator
deflateFilter.resetState();
LOG.info("Finished reseting deflator");
// re-generate data
data.reset();
generator = new RandomDatum.Generator(seed);
for (int i = 0; i < count; ++i) {
generator.next();
RandomDatum key = generator.getKey();
RandomDatum value = generator.getValue();
key.write(data);
value.write(data);
}
DataInputBuffer originalData = new DataInputBuffer();
DataInputStream originalIn = new DataInputStream(
new BufferedInputStream(originalData));
originalData.reset(data.getData(), 0, data.getLength());
// re-compress data
compressedDataBuffer.reset();
deflateOut = new DataOutputStream(new BufferedOutputStream(
compressedDataBuffer));
deflateFilter = codec.createOutputStream(deflateOut);
deflateFilter.write(data.getData(), 0, data.getLength());
deflateFilter.finish();
deflateFilter.flush();
LOG.info("Finished re-compressing data");
// De-compress data
DataInputBuffer deCompressedDataBuffer = new DataInputBuffer();
deCompressedDataBuffer.reset(compressedDataBuffer.getData(), 0,
compressedDataBuffer.getLength());
CompressionInputStream inflateFilter = codec
.createInputStream(deCompressedDataBuffer);
DataInputStream inflateIn = new DataInputStream(
new BufferedInputStream(inflateFilter));
// Check
for (int i = 0; i < count; ++i) {
RandomDatum k1 = new RandomDatum();
RandomDatum v1 = new RandomDatum();
k1.readFields(originalIn);
v1.readFields(originalIn);
RandomDatum k2 = new RandomDatum();
RandomDatum v2 = new RandomDatum();
k2.readFields(inflateIn);
v2.readFields(inflateIn);
assertTrue(
"original and compressed-then-decompressed-output not equal",
k1.equals(k2) && v1.equals(v2));
}
LOG.info("SUCCESS! Completed checking " + count + " records");
}
}

View File

@ -1,3 +1,19 @@
#!/bin/sh
ps -ef | grep apacheds | grep -v grep | cut -f4 -d ' ' |xargs kill -9
# 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.
ps -ef | grep apacheds | grep -v grep | awk '{printf $2"\n"}' | xargs -t --no-run-if-empty kill -9

View File

@ -172,6 +172,11 @@ Trunk (Unreleased)
HDFS-4234. Use generic code for choosing datanode in Balancer. (szetszwo)
HDFS-4334. Add a unique id to INode. (Brandon Li via szetszwo)
HDFS-4346. Add SequentialNumber as a base class for INodeId and
GenerationStamp. (szetszwo)
OPTIMIZATIONS
BUG FIXES
@ -280,12 +285,17 @@ Trunk (Unreleased)
HDFS-4310. fix test org.apache.hadoop.hdfs.server.datanode.
TestStartSecureDataNode (Ivan A. Veselovsky via atm)
HDFS-4274. BlockPoolSliceScanner does not close verification log during
shutdown. (Chris Nauroth via suresh)
HDFS-4275. MiniDFSCluster-based tests fail on Windows due to failure
to delete test namenode directory. (Chris Nauroth via suresh)
HDFS-4338. TestNameNodeMetrics#testCorruptBlock is flaky. (Andrew Wang via
atm)
HDFS-4261. Fix bugs in Balaner causing infinite loop and
TestBalancerWithNodeGroup timeing out. (Junping Du via szetszwo)
HDFS-4382. Fix typo MAX_NOT_CHANGED_INTERATIONS. (Ted Yu via suresh)
Release 2.0.3-alpha - Unreleased
INCOMPATIBLE CHANGES
@ -293,6 +303,12 @@ Release 2.0.3-alpha - Unreleased
HDFS-4122. Cleanup HDFS logs and reduce the size of logged messages.
(suresh)
HDFS-4362. GetDelegationTokenResponseProto does not handle null token.
(suresh)
HDFS-4367. GetDataEncryptionKeyResponseProto does not handle null
response. (suresh)
NEW FEATURES
HDFS-2656. Add libwebhdfs, a pure C client based on WebHDFS.
@ -432,6 +448,35 @@ Release 2.0.3-alpha - Unreleased
HDFS-4326. bump up Tomcat version for HttpFS to 6.0.36. (tucu via acmurthy)
HDFS-4270. Introduce soft and hard limits for max replication so that
replications of the highest priority are allowed to choose a source datanode
that has reached its soft limit but not the hard limit. (Derek Dagit via
szetszwo)
HADOOP-9173. Add security token protobuf definition to common and
use it in hdfs. (suresh)
HDFS-4030. BlockManager excessBlocksCount and
postponedMisreplicatedBlocksCount should be AtomicLongs. (eli)
HDFS-4031. Update findbugsExcludeFile.xml to include findbugs 2
exclusions. (eli)
HDFS-4033. Miscellaneous findbugs 2 fixes. (eli)
HDFS-4034. Remove redundant null checks. (eli)
HDFS-4035. LightWeightGSet and LightWeightHashSet increment a
volatile without synchronization. (eli)
HDFS-4032. Specify the charset explicitly rather than rely on the
default. (eli)
HDFS-4363. Combine PBHelper and HdfsProtoUtil and remove redundant
methods. (suresh)
HDFS-4377. Some trivial DN comment cleanup. (eli)
HDFS-4381. Document fsimage format details in FSImageFormat class javadoc.
(Jing Zhao via suresh)
@ -630,6 +675,34 @@ Release 2.0.3-alpha - Unreleased
HDFS-4140. fuse-dfs handles open(O_TRUNC) poorly. (Colin Patrick McCabe
via atm)
HDFS-4308. addBlock() should persist file blocks once.
(Plamen Jeliazkov via shv)
HDFS-4347. Avoid infinite waiting checkpoint to complete in TestBackupNode.
(Plamen Jeliazkov via shv)
HDFS-4349. Add test for reading files from BackupNode. (shv)
HDFS-4302. Fix fatal exception when starting NameNode with DEBUG logs
(Eugene Koontz via todd)
HDFS-3970. Fix bug causing rollback of HDFS upgrade to result in bad
VERSION file. (Vinay and Andrew Wang via atm)
HDFS-4306. PBHelper.convertLocatedBlock miss convert BlockToken. (Binglin
Chang via atm)
HDFS-4384. test_libhdfs_threaded gets SEGV if JNIEnv cannot be
initialized. (Colin Patrick McCabe via eli)
HDFS-4328. TestLargeBlock#testLargeBlockSize is timing out. (Chris Nauroth
via atm)
HDFS-4274. BlockPoolSliceScanner does not close verification log during
shutdown. (Chris Nauroth via suresh)
HDFS-1245. Pluggable block id generation. (shv)
BREAKDOWN OF HDFS-3077 SUBTASKS
HDFS-3077. Quorum-based protocol for reading and writing edit logs.
@ -732,6 +805,8 @@ Release 2.0.3-alpha - Unreleased
HDFS-4017. Unclosed FileInputStream in GetJournalEditServlet
(Chao Shi via todd)
HDFS-4351. In BlockPlacementPolicyDefault.chooseTarget(..), numOfReplicas
needs to be updated when avoiding stale nodes. (Andrew Wang via szetszwo)
Release 2.0.2-alpha - 2012-09-07

View File

@ -290,4 +290,38 @@
<Method name="persistPaxosData" />
<Bug pattern="OS_OPEN_STREAM" />
</Match>
<!-- Don't complain about LocalDatanodeInfo's anonymous class -->
<Match>
<Class name="org.apache.hadoop.hdfs.BlockReaderLocal$LocalDatanodeInfo$1" />
<Bug pattern="SE_BAD_FIELD_INNER_CLASS" />
</Match>
<!-- Only one method increments numFailedVolumes and it is synchronized -->
<Match>
<Class name="org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeList" />
<Field name="numFailedVolumes" />
<Bug pattern="VO_VOLATILE_INCREMENT" />
</Match>
<!-- Access to pendingReceivedRequests is synchronized -->
<Match>
<Class name="org.apache.hadoop.hdfs.server.datanode.BPServiceActor" />
<Method name="notifyNamenodeBlockImmediately" />
<Field name="pendingReceivedRequests" />
<Bug pattern="VO_VOLATILE_INCREMENT" />
</Match>
<!-- The "LightWeight" classes are explicitly not thread safe -->
<Match>
<Class name="org.apache.hadoop.hdfs.util.LightWeightGSet" />
<Field name="modification" />
<Bug pattern="VO_VOLATILE_INCREMENT" />
</Match>
<Match>
<Class name="org.apache.hadoop.hdfs.util.LightWeightHashSet" />
<Field name="modification" />
<Bug pattern="VO_VOLATILE_INCREMENT" />
</Match>
<Match>
<Class name="org.apache.hadoop.hdfs.util.LightWeightLinkedSet" />
<Field name="modification" />
<Bug pattern="VO_VOLATILE_INCREMENT" />
</Match>
</FindBugsFilter>

View File

@ -420,8 +420,10 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
<configuration>
<executable>protoc</executable>
<arguments>
<argument>-I../../hadoop-common-project/hadoop-common/src/main/proto/</argument>
<argument>-Isrc/main/proto/</argument>
<argument>--java_out=target/generated-sources/java</argument>
<argument>src/main/proto/hdfs.proto</argument>
<argument>src/main/proto/GetUserMappingsProtocol.proto</argument>
<argument>src/main/proto/HAZKInfo.proto</argument>
<argument>src/main/proto/InterDatanodeProtocol.proto</argument>
@ -429,7 +431,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
<argument>src/main/proto/RefreshAuthorizationPolicyProtocol.proto</argument>
<argument>src/main/proto/RefreshUserMappingsProtocol.proto</argument>
<argument>src/main/proto/datatransfer.proto</argument>
<argument>src/main/proto/hdfs.proto</argument>
</arguments>
</configuration>
</execution>
@ -442,6 +443,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
<configuration>
<executable>protoc</executable>
<arguments>
<argument>-I../../hadoop-common-project/hadoop-common/src/main/proto/</argument>
<argument>-Isrc/main/proto/</argument>
<argument>--java_out=target/generated-sources/java</argument>
<argument>src/main/proto/ClientDatanodeProtocol.proto</argument>
@ -458,6 +460,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
<configuration>
<executable>protoc</executable>
<arguments>
<argument>-I../../hadoop-common-project/hadoop-common/src/main/proto/</argument>
<argument>-Isrc/main/proto/</argument>
<argument>--java_out=target/generated-sources/java</argument>
<argument>src/main/proto/ClientNamenodeProtocol.proto</argument>
@ -474,6 +477,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
<configuration>
<executable>protoc</executable>
<arguments>
<argument>-I../../hadoop-common-project/hadoop-common/src/main/proto/</argument>
<argument>-Isrc/main/proto/</argument>
<argument>--java_out=target/generated-sources/java</argument>
<argument>src/main/proto/QJournalProtocol.proto</argument>

View File

@ -143,6 +143,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
<configuration>
<executable>protoc</executable>
<arguments>
<argument>-I../../../../../hadoop-common-project/hadoop-common/src/main/proto/</argument>
<argument>-Isrc/main/proto/</argument>
<argument>-I../../main/proto</argument>
<argument>--java_out=target/generated-sources/java</argument>

View File

@ -114,7 +114,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
@ -128,6 +127,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
@ -364,7 +364,7 @@ public class DFSClient implements java.io.Closeable {
/**
* Same as this(nameNodeUri, conf, null);
* @see #DFSClient(InetSocketAddress, Configuration, org.apache.hadoop.fs.FileSystem.Statistics)
* @see #DFSClient(URI, Configuration, FileSystem.Statistics)
*/
public DFSClient(URI nameNodeUri, Configuration conf
) throws IOException {
@ -373,7 +373,7 @@ public class DFSClient implements java.io.Closeable {
/**
* Same as this(nameNodeUri, null, conf, stats);
* @see #DFSClient(InetSocketAddress, ClientProtocol, Configuration, org.apache.hadoop.fs.FileSystem.Statistics)
* @see #DFSClient(URI, ClientProtocol, Configuration, FileSystem.Statistics)
*/
public DFSClient(URI nameNodeUri, Configuration conf,
FileSystem.Statistics stats)
@ -1158,8 +1158,8 @@ public class DFSClient implements java.io.Closeable {
/**
* Call {@link #create(String, FsPermission, EnumSet, short, long,
* Progressable, int)} with default <code>permission</code>
* {@link FsPermission#getDefault()}.
* Progressable, int, ChecksumOpt)} with default <code>permission</code>
* {@link FsPermission#getFileDefault()}.
*
* @param src File name
* @param overwrite overwrite an existing file if true
@ -1177,7 +1177,7 @@ public class DFSClient implements java.io.Closeable {
Progressable progress,
int buffersize)
throws IOException {
return create(src, FsPermission.getDefault(),
return create(src, FsPermission.getFileDefault(),
overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
: EnumSet.of(CreateFlag.CREATE), replication, blockSize, progress,
buffersize, null);
@ -1208,7 +1208,7 @@ public class DFSClient implements java.io.Closeable {
*
* @param src File name
* @param permission The permission of the directory being created.
* If null, use default permission {@link FsPermission#getDefault()}
* If null, use default permission {@link FsPermission#getFileDefault()}
* @param flag indicates create a new file or create/overwrite an
* existing file or append to an existing file
* @param createParent create missing parent directory if true
@ -1234,7 +1234,7 @@ public class DFSClient implements java.io.Closeable {
ChecksumOpt checksumOpt) throws IOException {
checkOpen();
if (permission == null) {
permission = FsPermission.getDefault();
permission = FsPermission.getFileDefault();
}
FsPermission masked = permission.applyUMask(dfsClientConf.uMask);
if(LOG.isDebugEnabled()) {
@ -1269,7 +1269,7 @@ public class DFSClient implements java.io.Closeable {
/**
* Same as {{@link #create(String, FsPermission, EnumSet, short, long,
* Progressable, int)} except that the permission
* Progressable, int, ChecksumOpt)} except that the permission
* is absolute (ie has already been masked with umask.
*/
public DFSOutputStream primitiveCreate(String src,
@ -1460,7 +1460,7 @@ public class DFSClient implements java.io.Closeable {
}
/**
* Delete file or directory.
* See {@link ClientProtocol#delete(String)}.
* See {@link ClientProtocol#delete(String, boolean)}.
*/
@Deprecated
public boolean delete(String src) throws IOException {
@ -1600,8 +1600,7 @@ public class DFSClient implements java.io.Closeable {
if (shouldEncryptData()) {
synchronized (this) {
if (encryptionKey == null ||
(encryptionKey != null &&
encryptionKey.expiryDate < Time.now())) {
encryptionKey.expiryDate < Time.now()) {
LOG.debug("Getting new encryption token from NN");
encryptionKey = namenode.getDataEncryptionKey();
}
@ -1687,7 +1686,7 @@ public class DFSClient implements java.io.Closeable {
new Sender(out).blockChecksum(block, lb.getBlockToken());
final BlockOpResponseProto reply =
BlockOpResponseProto.parseFrom(HdfsProtoUtil.vintPrefixed(in));
BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in));
if (reply.getStatus() != Status.SUCCESS) {
if (reply.getStatus() == Status.ERROR_ACCESS_TOKEN
@ -1734,8 +1733,8 @@ public class DFSClient implements java.io.Closeable {
md5.write(md5out);
// read crc-type
final DataChecksum.Type ct = HdfsProtoUtil.
fromProto(checksumData.getCrcType());
final DataChecksum.Type ct = PBHelper.convert(checksumData
.getCrcType());
if (i == 0) { // first block
crcType = ct;
} else if (crcType != DataChecksum.Type.MIXED
@ -1899,7 +1898,7 @@ public class DFSClient implements java.io.Closeable {
* @param isChecked
* If true, then check only active namenode's safemode status, else
* check first namenode's status.
* @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeActio,boolean)
* @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction, boolean)
*/
public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException{
return namenode.setSafeMode(action, isChecked);

View File

@ -143,6 +143,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final int DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_DEFAULT = -1;
public static final String DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY = "dfs.namenode.replication.max-streams";
public static final int DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT = 2;
public static final String DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY = "dfs.namenode.replication.max-streams-hard-limit";
public static final int DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_DEFAULT = 4;
public static final String DFS_WEBHDFS_ENABLED_KEY = "dfs.webhdfs.enabled";
public static final boolean DFS_WEBHDFS_ENABLED_DEFAULT = false;
public static final String DFS_PERMISSIONS_ENABLED_KEY = "dfs.permissions.enabled";

View File

@ -52,7 +52,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
@ -66,6 +65,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
@ -884,7 +884,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
//ack
BlockOpResponseProto response =
BlockOpResponseProto.parseFrom(HdfsProtoUtil.vintPrefixed(in));
BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in));
if (SUCCESS != response.getStatus()) {
throw new IOException("Failed to add a datanode");
}
@ -1074,7 +1074,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
// receive ack for connect
BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
HdfsProtoUtil.vintPrefixed(blockReplyStream));
PBHelper.vintPrefixed(blockReplyStream));
pipelineStatus = resp.getStatus();
firstBadLink = resp.getFirstBadLink();

View File

@ -80,6 +80,7 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.ToolRunner;
import com.google.common.base.Charsets;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
@ -222,12 +223,7 @@ public class DFSUtil {
* Converts a string to a byte array using UTF8 encoding.
*/
public static byte[] string2Bytes(String str) {
try {
return str.getBytes("UTF8");
} catch(UnsupportedEncodingException e) {
assert false : "UTF8 encoding is not supported ";
}
return null;
return str.getBytes(Charsets.UTF_8);
}
/**
@ -240,18 +236,14 @@ public class DFSUtil {
&& (pathComponents[0] == null || pathComponents[0].length == 0)) {
return Path.SEPARATOR;
}
try {
StringBuilder result = new StringBuilder();
for (int i = 0; i < pathComponents.length; i++) {
result.append(new String(pathComponents[i], "UTF-8"));
if (i < pathComponents.length - 1) {
result.append(Path.SEPARATOR_CHAR);
}
StringBuilder result = new StringBuilder();
for (int i = 0; i < pathComponents.length; i++) {
result.append(new String(pathComponents[i], Charsets.UTF_8));
if (i < pathComponents.length - 1) {
result.append(Path.SEPARATOR_CHAR);
}
return result.toString();
} catch (UnsupportedEncodingException ex) {
throw new AssertionError("UTF-8 encoding is not supported.");
}
return result.toString();
}
/** Convert an object representing a path to a string. */

View File

@ -17,8 +17,6 @@
*/
package org.apache.hadoop.hdfs;
import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.DataInputStream;
@ -39,6 +37,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.io.IOUtils;
@ -392,7 +391,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
bufferSize));
BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
vintPrefixed(in));
PBHelper.vintPrefixed(in));
RemoteBlockReader2.checkSuccess(status, sock, block, file);
ReadOpChecksumInfoProto checksumInfo =
status.getReadOpChecksumInfo();

View File

@ -17,8 +17,6 @@
*/
package org.apache.hadoop.hdfs;
import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
import java.io.BufferedOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
@ -43,6 +41,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatus
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
import org.apache.hadoop.net.SocketInputWrapper;
@ -401,7 +400,7 @@ public class RemoteBlockReader2 implements BlockReader {
DataInputStream in = new DataInputStream(ioStreams.in);
BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
vintPrefixed(in));
PBHelper.vintPrefixed(in));
checkSuccess(status, sock, block, file);
ReadOpChecksumInfoProto checksumInfo =
status.getReadOpChecksumInfo();

View File

@ -67,7 +67,10 @@ public class HdfsFileStatus {
this.modification_time = modification_time;
this.access_time = access_time;
this.permission = (permission == null) ?
FsPermission.getDefault() : permission;
((isdir || symlink!=null) ?
FsPermission.getDefault() :
FsPermission.getFileDefault()) :
permission;
this.owner = (owner == null) ? "" : owner;
this.group = (group == null) ? "" : group;
this.symlink = symlink;

View File

@ -1,179 +0,0 @@
/**
* 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.protocol;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.util.ExactSizeInputStream;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.security.token.Token;
import com.google.common.collect.Lists;
import com.google.protobuf.ByteString;
import com.google.protobuf.CodedInputStream;
/**
* Utilities for converting to and from protocol buffers used in the
* HDFS wire protocol, as well as some generic utilities useful
* for dealing with protocol buffers.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public abstract class HdfsProtoUtil {
//// Block Token ////
public static HdfsProtos.BlockTokenIdentifierProto toProto(Token<?> blockToken) {
return HdfsProtos.BlockTokenIdentifierProto.newBuilder()
.setIdentifier(ByteString.copyFrom(blockToken.getIdentifier()))
.setPassword(ByteString.copyFrom(blockToken.getPassword()))
.setKind(blockToken.getKind().toString())
.setService(blockToken.getService().toString())
.build();
}
public static Token<BlockTokenIdentifier> fromProto(HdfsProtos.BlockTokenIdentifierProto proto) {
return new Token<BlockTokenIdentifier>(proto.getIdentifier().toByteArray(),
proto.getPassword().toByteArray(),
new Text(proto.getKind()),
new Text(proto.getService()));
}
//// Extended Block ////
public static HdfsProtos.ExtendedBlockProto toProto(ExtendedBlock block) {
return HdfsProtos.ExtendedBlockProto.newBuilder()
.setBlockId(block.getBlockId())
.setPoolId(block.getBlockPoolId())
.setNumBytes(block.getNumBytes())
.setGenerationStamp(block.getGenerationStamp())
.build();
}
public static ExtendedBlock fromProto(HdfsProtos.ExtendedBlockProto proto) {
return new ExtendedBlock(
proto.getPoolId(), proto.getBlockId(),
proto.getNumBytes(), proto.getGenerationStamp());
}
//// DatanodeID ////
private static HdfsProtos.DatanodeIDProto toProto(
DatanodeID dni) {
return HdfsProtos.DatanodeIDProto.newBuilder()
.setIpAddr(dni.getIpAddr())
.setHostName(dni.getHostName())
.setStorageID(dni.getStorageID())
.setXferPort(dni.getXferPort())
.setInfoPort(dni.getInfoPort())
.setIpcPort(dni.getIpcPort())
.build();
}
private static DatanodeID fromProto(HdfsProtos.DatanodeIDProto idProto) {
return new DatanodeID(
idProto.getIpAddr(),
idProto.getHostName(),
idProto.getStorageID(),
idProto.getXferPort(),
idProto.getInfoPort(),
idProto.getIpcPort());
}
//// DatanodeInfo ////
public static HdfsProtos.DatanodeInfoProto toProto(DatanodeInfo dni) {
return HdfsProtos.DatanodeInfoProto.newBuilder()
.setId(toProto((DatanodeID)dni))
.setCapacity(dni.getCapacity())
.setDfsUsed(dni.getDfsUsed())
.setRemaining(dni.getRemaining())
.setBlockPoolUsed(dni.getBlockPoolUsed())
.setLastUpdate(dni.getLastUpdate())
.setXceiverCount(dni.getXceiverCount())
.setLocation(dni.getNetworkLocation())
.setAdminState(HdfsProtos.DatanodeInfoProto.AdminState.valueOf(
dni.getAdminState().name()))
.build();
}
public static DatanodeInfo fromProto(HdfsProtos.DatanodeInfoProto dniProto) {
DatanodeInfo dniObj = new DatanodeInfo(fromProto(dniProto.getId()),
dniProto.getLocation());
dniObj.setCapacity(dniProto.getCapacity());
dniObj.setDfsUsed(dniProto.getDfsUsed());
dniObj.setRemaining(dniProto.getRemaining());
dniObj.setBlockPoolUsed(dniProto.getBlockPoolUsed());
dniObj.setLastUpdate(dniProto.getLastUpdate());
dniObj.setXceiverCount(dniProto.getXceiverCount());
dniObj.setAdminState(DatanodeInfo.AdminStates.valueOf(
dniProto.getAdminState().name()));
return dniObj;
}
public static ArrayList<? extends HdfsProtos.DatanodeInfoProto> toProtos(
DatanodeInfo[] dnInfos, int startIdx) {
ArrayList<HdfsProtos.DatanodeInfoProto> protos =
Lists.newArrayListWithCapacity(dnInfos.length);
for (int i = startIdx; i < dnInfos.length; i++) {
protos.add(toProto(dnInfos[i]));
}
return protos;
}
public static DatanodeInfo[] fromProtos(
List<HdfsProtos.DatanodeInfoProto> targetsList) {
DatanodeInfo[] ret = new DatanodeInfo[targetsList.size()];
int i = 0;
for (HdfsProtos.DatanodeInfoProto proto : targetsList) {
ret[i++] = fromProto(proto);
}
return ret;
}
public static DataChecksum.Type fromProto(HdfsProtos.ChecksumTypeProto type) {
return DataChecksum.Type.valueOf(type.getNumber());
}
public static HdfsProtos.ChecksumTypeProto toProto(DataChecksum.Type type) {
return HdfsProtos.ChecksumTypeProto.valueOf(type.id);
}
public static InputStream vintPrefixed(final InputStream input)
throws IOException {
final int firstByte = input.read();
if (firstByte == -1) {
throw new EOFException("Premature EOF: no length prefix available");
}
int size = CodedInputStream.readRawVarint32(firstByte, input);
assert size >= 0;
return new ExactSizeInputStream(input, size);
}
}

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hdfs.protocol.datatransfer;
import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed;
import java.io.DataInputStream;
import java.io.DataOutputStream;
@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
import org.apache.hadoop.security.SaslInputStream;
import org.apache.hadoop.security.SaslOutputStream;
import com.google.common.base.Charsets;
import com.google.common.collect.Maps;
import com.google.protobuf.ByteString;
@ -399,7 +400,7 @@ public class DataTransferEncryptor {
DataEncryptionKey encryptionKey) {
return encryptionKey.keyId + NAME_DELIMITER +
encryptionKey.blockPoolId + NAME_DELIMITER +
new String(Base64.encodeBase64(encryptionKey.nonce, false));
new String(Base64.encodeBase64(encryptionKey.nonce, false), Charsets.UTF_8);
}
/**
@ -427,7 +428,7 @@ public class DataTransferEncryptor {
}
private static char[] encryptionKeyToPassword(byte[] encryptionKey) {
return new String(Base64.encodeBase64(encryptionKey, false)).toCharArray();
return new String(Base64.encodeBase64(encryptionKey, false), Charsets.UTF_8).toCharArray();
}
/**

View File

@ -21,12 +21,12 @@ package org.apache.hadoop.hdfs.protocol.datatransfer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ChecksumTypeProto;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.DataChecksum;
@ -41,23 +41,17 @@ import org.apache.hadoop.util.DataChecksum;
public abstract class DataTransferProtoUtil {
static BlockConstructionStage fromProto(
OpWriteBlockProto.BlockConstructionStage stage) {
return BlockConstructionStage.valueOf(BlockConstructionStage.class,
stage.name());
return BlockConstructionStage.valueOf(stage.name());
}
static OpWriteBlockProto.BlockConstructionStage toProto(
BlockConstructionStage stage) {
return OpWriteBlockProto.BlockConstructionStage.valueOf(
stage.name());
return OpWriteBlockProto.BlockConstructionStage.valueOf(stage.name());
}
public static ChecksumProto toProto(DataChecksum checksum) {
ChecksumTypeProto type = HdfsProtoUtil.toProto(checksum.getChecksumType());
if (type == null) {
throw new IllegalArgumentException(
"Can't convert checksum to protobuf: " + checksum);
}
ChecksumTypeProto type = PBHelper.convert(checksum.getChecksumType());
// ChecksumType#valueOf never returns null
return ChecksumProto.newBuilder()
.setBytesPerChecksum(checksum.getBytesPerChecksum())
.setType(type)
@ -68,8 +62,7 @@ public abstract class DataTransferProtoUtil {
if (proto == null) return null;
int bytesPerChecksum = proto.getBytesPerChecksum();
DataChecksum.Type type = HdfsProtoUtil.fromProto(proto.getType());
DataChecksum.Type type = PBHelper.convert(proto.getType());
return DataChecksum.newDataChecksum(type, bytesPerChecksum);
}
@ -86,8 +79,8 @@ public abstract class DataTransferProtoUtil {
static BaseHeaderProto buildBaseHeader(ExtendedBlock blk,
Token<BlockTokenIdentifier> blockToken) {
return BaseHeaderProto.newBuilder()
.setBlock(HdfsProtoUtil.toProto(blk))
.setToken(HdfsProtoUtil.toProto(blockToken))
.setBlock(PBHelper.convert(blk))
.setToken(PBHelper.convert(blockToken))
.build();
}
}

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hdfs.protocol.datatransfer;
import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed;
import java.io.IOException;
import java.io.InputStream;

View File

@ -17,9 +17,7 @@
*/
package org.apache.hadoop.hdfs.protocol.datatransfer;
import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.fromProto;
import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.fromProtos;
import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed;
import static org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.fromProto;
import java.io.DataInputStream;
@ -33,6 +31,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
/** Receiver */
@InterfaceAudience.Private
@ -85,8 +84,8 @@ public abstract class Receiver implements DataTransferProtocol {
/** Receive OP_READ_BLOCK */
private void opReadBlock() throws IOException {
OpReadBlockProto proto = OpReadBlockProto.parseFrom(vintPrefixed(in));
readBlock(fromProto(proto.getHeader().getBaseHeader().getBlock()),
fromProto(proto.getHeader().getBaseHeader().getToken()),
readBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()),
PBHelper.convert(proto.getHeader().getBaseHeader().getToken()),
proto.getHeader().getClientName(),
proto.getOffset(),
proto.getLen());
@ -95,11 +94,11 @@ public abstract class Receiver implements DataTransferProtocol {
/** Receive OP_WRITE_BLOCK */
private void opWriteBlock(DataInputStream in) throws IOException {
final OpWriteBlockProto proto = OpWriteBlockProto.parseFrom(vintPrefixed(in));
writeBlock(fromProto(proto.getHeader().getBaseHeader().getBlock()),
fromProto(proto.getHeader().getBaseHeader().getToken()),
writeBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()),
PBHelper.convert(proto.getHeader().getBaseHeader().getToken()),
proto.getHeader().getClientName(),
fromProtos(proto.getTargetsList()),
fromProto(proto.getSource()),
PBHelper.convert(proto.getTargetsList()),
PBHelper.convert(proto.getSource()),
fromProto(proto.getStage()),
proto.getPipelineSize(),
proto.getMinBytesRcvd(), proto.getMaxBytesRcvd(),
@ -111,33 +110,33 @@ public abstract class Receiver implements DataTransferProtocol {
private void opTransferBlock(DataInputStream in) throws IOException {
final OpTransferBlockProto proto =
OpTransferBlockProto.parseFrom(vintPrefixed(in));
transferBlock(fromProto(proto.getHeader().getBaseHeader().getBlock()),
fromProto(proto.getHeader().getBaseHeader().getToken()),
transferBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()),
PBHelper.convert(proto.getHeader().getBaseHeader().getToken()),
proto.getHeader().getClientName(),
fromProtos(proto.getTargetsList()));
PBHelper.convert(proto.getTargetsList()));
}
/** Receive OP_REPLACE_BLOCK */
private void opReplaceBlock(DataInputStream in) throws IOException {
OpReplaceBlockProto proto = OpReplaceBlockProto.parseFrom(vintPrefixed(in));
replaceBlock(fromProto(proto.getHeader().getBlock()),
fromProto(proto.getHeader().getToken()),
replaceBlock(PBHelper.convert(proto.getHeader().getBlock()),
PBHelper.convert(proto.getHeader().getToken()),
proto.getDelHint(),
fromProto(proto.getSource()));
PBHelper.convert(proto.getSource()));
}
/** Receive OP_COPY_BLOCK */
private void opCopyBlock(DataInputStream in) throws IOException {
OpCopyBlockProto proto = OpCopyBlockProto.parseFrom(vintPrefixed(in));
copyBlock(fromProto(proto.getHeader().getBlock()),
fromProto(proto.getHeader().getToken()));
copyBlock(PBHelper.convert(proto.getHeader().getBlock()),
PBHelper.convert(proto.getHeader().getToken()));
}
/** Receive OP_BLOCK_CHECKSUM */
private void opBlockChecksum(DataInputStream in) throws IOException {
OpBlockChecksumProto proto = OpBlockChecksumProto.parseFrom(vintPrefixed(in));
blockChecksum(fromProto(proto.getHeader().getBlock()),
fromProto(proto.getHeader().getToken()));
blockChecksum(PBHelper.convert(proto.getHeader().getBlock()),
PBHelper.convert(proto.getHeader().getToken()));
}
}

View File

@ -17,8 +17,6 @@
*/
package org.apache.hadoop.hdfs.protocol.datatransfer;
import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.toProto;
import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.toProtos;
import static org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.toProto;
import java.io.DataOutput;
@ -37,6 +35,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.DataChecksum;
@ -105,7 +104,7 @@ public class Sender implements DataTransferProtocol {
OpWriteBlockProto.Builder proto = OpWriteBlockProto.newBuilder()
.setHeader(header)
.addAllTargets(toProtos(targets, 1))
.addAllTargets(PBHelper.convert(targets, 1))
.setStage(toProto(stage))
.setPipelineSize(pipelineSize)
.setMinBytesRcvd(minBytesRcvd)
@ -114,7 +113,7 @@ public class Sender implements DataTransferProtocol {
.setRequestedChecksum(checksumProto);
if (source != null) {
proto.setSource(toProto(source));
proto.setSource(PBHelper.convertDatanodeInfo(source));
}
send(out, Op.WRITE_BLOCK, proto.build());
@ -129,7 +128,7 @@ public class Sender implements DataTransferProtocol {
OpTransferBlockProto proto = OpTransferBlockProto.newBuilder()
.setHeader(DataTransferProtoUtil.buildClientHeader(
blk, clientName, blockToken))
.addAllTargets(toProtos(targets, 0))
.addAllTargets(PBHelper.convert(targets))
.build();
send(out, Op.TRANSFER_BLOCK, proto);
@ -143,7 +142,7 @@ public class Sender implements DataTransferProtocol {
OpReplaceBlockProto proto = OpReplaceBlockProto.newBuilder()
.setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
.setDelHint(delHint)
.setSource(toProto(source))
.setSource(PBHelper.convertDatanodeInfo(source))
.build();
send(out, Op.REPLACE_BLOCK, proto);

View File

@ -37,9 +37,9 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetRep
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
import org.apache.hadoop.security.token.Token;
import com.google.protobuf.ByteString;
@ -133,7 +133,7 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
}
List<Token<BlockTokenIdentifier>> tokens =
new ArrayList<Token<BlockTokenIdentifier>>(request.getTokensCount());
for (BlockTokenIdentifierProto b : request.getTokensList()) {
for (TokenProto b : request.getTokensList()) {
tokens.add(PBHelper.convert(b));
}
// Call the real implementation

View File

@ -44,7 +44,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdf
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.ipc.ProtobufHelper;
@ -55,6 +54,7 @@ import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
import org.apache.hadoop.security.token.Token;
import com.google.protobuf.ByteString;
@ -222,8 +222,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
// Convert to proto objects
List<ExtendedBlockProto> blocksProtos =
new ArrayList<ExtendedBlockProto>(blocks.size());
List<BlockTokenIdentifierProto> tokensProtos =
new ArrayList<BlockTokenIdentifierProto>(tokens.size());
List<TokenProto> tokensProtos =
new ArrayList<TokenProto>(tokens.size());
for (ExtendedBlock b : blocks) {
blocksProtos.add(PBHelper.convert(b));
}

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hdfs.protocolPB;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
@ -138,13 +137,13 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.token.Token;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
@ -516,10 +515,10 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
RpcController controller, GetDatanodeReportRequestProto req)
throws ServiceException {
try {
DatanodeInfoProto[] result = PBHelper.convert(server
List<? extends DatanodeInfoProto> result = PBHelper.convert(server
.getDatanodeReport(PBHelper.convert(req.getType())));
return GetDatanodeReportResponseProto.newBuilder()
.addAllDi(Arrays.asList(result)).build();
.addAllDi(result).build();
} catch (IOException e) {
throw new ServiceException(e);
}
@ -798,10 +797,14 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
RpcController controller, GetDelegationTokenRequestProto req)
throws ServiceException {
try {
BlockTokenIdentifierProto result = PBHelper.convert(server
.getDelegationToken(new Text(req.getRenewer())));
return GetDelegationTokenResponseProto.newBuilder().setToken(result)
.build();
Token<DelegationTokenIdentifier> token = server
.getDelegationToken(new Text(req.getRenewer()));
GetDelegationTokenResponseProto.Builder rspBuilder =
GetDelegationTokenResponseProto.newBuilder();
if (token != null) {
rspBuilder.setToken(PBHelper.convert(token));
}
return rspBuilder.build();
} catch (IOException e) {
throw new ServiceException(e);
}
@ -859,10 +862,13 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
RpcController controller, GetDataEncryptionKeyRequestProto request)
throws ServiceException {
try {
GetDataEncryptionKeyResponseProto.Builder builder =
GetDataEncryptionKeyResponseProto.newBuilder();
DataEncryptionKey encryptionKey = server.getDataEncryptionKey();
return GetDataEncryptionKeyResponseProto.newBuilder()
.setDataEncryptionKey(PBHelper.convert(encryptionKey))
.build();
if (encryptionKey != null) {
builder.setDataEncryptionKey(PBHelper.convert(encryptionKey));
}
return builder.build();
} catch (IOException e) {
throw new ServiceException(e);
}

View File

@ -68,8 +68,10 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlo
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDelegationTokenRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDelegationTokenResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto;
@ -288,7 +290,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
if (previous != null)
req.setPrevious(PBHelper.convert(previous));
if (excludeNodes != null)
req.addAllExcludeNodes(Arrays.asList(PBHelper.convert(excludeNodes)));
req.addAllExcludeNodes(PBHelper.convert(excludeNodes));
try {
return PBHelper.convert(rpcProxy.addBlock(null, req.build()).getBlock());
} catch (ServiceException e) {
@ -306,8 +308,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
.newBuilder()
.setSrc(src)
.setBlk(PBHelper.convert(blk))
.addAllExistings(Arrays.asList(PBHelper.convert(existings)))
.addAllExcludes(Arrays.asList(PBHelper.convert(excludes)))
.addAllExistings(PBHelper.convert(existings))
.addAllExcludes(PBHelper.convert(excludes))
.setNumAdditionalNodes(numAdditionalNodes)
.setClientName(clientName)
.build();
@ -766,7 +768,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
.setRenewer(renewer.toString())
.build();
try {
return PBHelper.convertDelegationToken(rpcProxy.getDelegationToken(null, req).getToken());
GetDelegationTokenResponseProto resp = rpcProxy.getDelegationToken(null, req);
return resp.hasToken() ? PBHelper.convertDelegationToken(resp.getToken())
: null;
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
@ -823,8 +827,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
GetDataEncryptionKeyRequestProto req = GetDataEncryptionKeyRequestProto
.newBuilder().build();
try {
return PBHelper.convert(rpcProxy.getDataEncryptionKey(null, req)
.getDataEncryptionKey());
GetDataEncryptionKeyResponseProto rsp =
rpcProxy.getDataEncryptionKey(null, req);
return rsp.hasDataEncryptionKey() ?
PBHelper.convert(rsp.getDataEncryptionKey()) : null;
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}

View File

@ -17,6 +17,9 @@
*/
package org.apache.hadoop.hdfs.protocolPB;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.EnumSet;
@ -40,11 +43,11 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
@ -65,7 +68,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageRepor
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DataEncryptionKeyProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlocksWithLocationsProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointCommandProto;
@ -131,14 +133,20 @@ import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStat
import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
import org.apache.hadoop.hdfs.util.ExactSizeInputStream;
import org.apache.hadoop.io.EnumSetWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.DataChecksum;
import com.google.common.collect.Lists;
import com.google.protobuf.ByteString;
import com.google.protobuf.CodedInputStream;
/**
* Utilities for converting protobuf classes to and from implementation classes.
* Utilities for converting protobuf classes to and from implementation classes
* and other helper utilities to help in dealing with protobuf.
*
* Note that when converting from an internal type to protobuf type, the
* converter never return null for protobuf type. The check for internal type
@ -222,7 +230,8 @@ public class PBHelper {
// Arrays of DatanodeId
public static DatanodeIDProto[] convert(DatanodeID[] did) {
if (did == null) return null;
if (did == null)
return null;
final int len = did.length;
DatanodeIDProto[] result = new DatanodeIDProto[len];
for (int i = 0; i < len; ++i) {
@ -485,14 +494,26 @@ public class PBHelper {
}
return result;
}
public static List<? extends HdfsProtos.DatanodeInfoProto> convert(
DatanodeInfo[] dnInfos) {
return convert(dnInfos, 0);
}
static public DatanodeInfoProto[] convert(DatanodeInfo[] di) {
if (di == null) return null;
DatanodeInfoProto[] result = new DatanodeInfoProto[di.length];
for (int i = 0; i < di.length; i++) {
result[i] = PBHelper.convertDatanodeInfo(di[i]);
/**
* Copy from {@code dnInfos} to a target of list of same size starting at
* {@code startIdx}.
*/
public static List<? extends HdfsProtos.DatanodeInfoProto> convert(
DatanodeInfo[] dnInfos, int startIdx) {
if (dnInfos == null)
return null;
ArrayList<HdfsProtos.DatanodeInfoProto> protos = Lists
.newArrayListWithCapacity(dnInfos.length);
for (int i = startIdx; i < dnInfos.length; i++) {
protos.add(convert(dnInfos[i]));
}
return result;
return protos;
}
public static DatanodeInfo[] convert(List<DatanodeInfoProto> list) {
@ -555,8 +576,8 @@ public class PBHelper {
return lb;
}
public static BlockTokenIdentifierProto convert(Token<?> tok) {
return BlockTokenIdentifierProto.newBuilder().
public static TokenProto convert(Token<?> tok) {
return TokenProto.newBuilder().
setIdentifier(ByteString.copyFrom(tok.getIdentifier())).
setPassword(ByteString.copyFrom(tok.getPassword())).
setKind(tok.getKind().toString()).
@ -564,7 +585,7 @@ public class PBHelper {
}
public static Token<BlockTokenIdentifier> convert(
BlockTokenIdentifierProto blockToken) {
TokenProto blockToken) {
return new Token<BlockTokenIdentifier>(blockToken.getIdentifier()
.toByteArray(), blockToken.getPassword().toByteArray(), new Text(
blockToken.getKind()), new Text(blockToken.getService()));
@ -572,7 +593,7 @@ public class PBHelper {
public static Token<DelegationTokenIdentifier> convertDelegationToken(
BlockTokenIdentifierProto blockToken) {
TokenProto blockToken) {
return new Token<DelegationTokenIdentifier>(blockToken.getIdentifier()
.toByteArray(), blockToken.getPassword().toByteArray(), new Text(
blockToken.getKind()), new Text(blockToken.getService()));
@ -682,6 +703,8 @@ public class PBHelper {
case DatanodeProtocol.DNA_SHUTDOWN:
builder.setAction(BlockCommandProto.Action.SHUTDOWN);
break;
default:
throw new AssertionError("Invalid action");
}
Block[] blocks = cmd.getBlocks();
for (int i = 0; i < blocks.length; i++) {
@ -695,7 +718,7 @@ public class PBHelper {
DatanodeInfosProto[] ret = new DatanodeInfosProto[targets.length];
for (int i = 0; i < targets.length; i++) {
ret[i] = DatanodeInfosProto.newBuilder()
.addAllDatanodes(Arrays.asList(PBHelper.convert(targets[i]))).build();
.addAllDatanodes(PBHelper.convert(targets[i])).build();
}
return Arrays.asList(ret);
}
@ -864,25 +887,14 @@ public class PBHelper {
// Located Block Arrays and Lists
public static LocatedBlockProto[] convertLocatedBlock(LocatedBlock[] lb) {
if (lb == null) return null;
final int len = lb.length;
LocatedBlockProto[] result = new LocatedBlockProto[len];
for (int i = 0; i < len; ++i) {
result[i] = PBHelper.convert(lb[i]);
}
return result;
return convertLocatedBlock2(Arrays.asList(lb)).toArray(
new LocatedBlockProto[lb.length]);
}
public static LocatedBlock[] convertLocatedBlock(LocatedBlockProto[] lb) {
if (lb == null) return null;
final int len = lb.length;
LocatedBlock[] result = new LocatedBlock[len];
for (int i = 0; i < len; ++i) {
result[i] = new LocatedBlock(
PBHelper.convert(lb[i].getB()),
PBHelper.convert(lb[i].getLocsList()),
lb[i].getOffset(), lb[i].getCorrupt());
}
return result;
return convertLocatedBlock(Arrays.asList(lb)).toArray(
new LocatedBlock[lb.length]);
}
public static List<LocatedBlock> convertLocatedBlock(
@ -964,7 +976,7 @@ public class PBHelper {
fs.getFileBufferSize(),
fs.getEncryptDataTransfer(),
fs.getTrashInterval(),
HdfsProtoUtil.fromProto(fs.getChecksumType()));
PBHelper.convert(fs.getChecksumType()));
}
public static FsServerDefaultsProto convert(FsServerDefaults fs) {
@ -977,7 +989,7 @@ public class PBHelper {
.setFileBufferSize(fs.getFileBufferSize())
.setEncryptDataTransfer(fs.getEncryptDataTransfer())
.setTrashInterval(fs.getTrashInterval())
.setChecksumType(HdfsProtoUtil.toProto(fs.getChecksumType()))
.setChecksumType(PBHelper.convert(fs.getChecksumType()))
.build();
}
@ -1381,4 +1393,24 @@ public class PBHelper {
return SnapshottableDirectoryListingProto.newBuilder()
.addAllSnapshottableDirListing(protoList).build();
}
public static DataChecksum.Type convert(HdfsProtos.ChecksumTypeProto type) {
return DataChecksum.Type.valueOf(type.getNumber());
}
public static HdfsProtos.ChecksumTypeProto convert(DataChecksum.Type type) {
return HdfsProtos.ChecksumTypeProto.valueOf(type.id);
}
public static InputStream vintPrefixed(final InputStream input)
throws IOException {
final int firstByte = input.read();
if (firstByte == -1) {
throw new EOFException("Premature EOF: no length prefix available");
}
int size = CodedInputStream.readRawVarint32(firstByte, input);
assert size >= 0;
return new ExactSizeInputStream(input, size);
}
}

View File

@ -58,6 +58,7 @@ import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.security.SecurityUtil;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
import com.google.common.base.Stopwatch;
import com.google.common.collect.ImmutableList;
@ -934,7 +935,7 @@ class Journal implements Closeable {
fos.write('\n');
// Write human-readable data after the protobuf. This is only
// to assist in debugging -- it's not parsed at all.
OutputStreamWriter writer = new OutputStreamWriter(fos);
OutputStreamWriter writer = new OutputStreamWriter(fos, Charsets.UTF_8);
writer.write(String.valueOf(newData));
writer.write('\n');

View File

@ -18,7 +18,8 @@
package org.apache.hadoop.hdfs.server.balancer;
import static com.google.common.base.Preconditions.checkArgument;
import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
@ -189,6 +190,7 @@ public class Balancer {
* balancing purpose at a datanode
*/
public static final int MAX_NUM_CONCURRENT_MOVES = 5;
private static final int MAX_NO_PENDING_BLOCK_ITERATIONS = 5;
private static final String USAGE = "Usage: java "
+ Balancer.class.getSimpleName()
@ -224,7 +226,6 @@ public class Balancer {
= new HashMap<String, BalancerDatanode>();
private NetworkTopology cluster;
final static private int MOVER_THREAD_POOL_SIZE = 1000;
final private ExecutorService moverExecutor =
Executors.newFixedThreadPool(MOVER_THREAD_POOL_SIZE);
@ -752,6 +753,7 @@ public class Balancer {
long startTime = Time.now();
this.blocksToReceive = 2*scheduledSize;
boolean isTimeUp = false;
int noPendingBlockIteration = 0;
while(!isTimeUp && scheduledSize>0 &&
(!srcBlockList.isEmpty() || blocksToReceive>0)) {
PendingBlockMove pendingBlock = chooseNextBlockToMove();
@ -775,7 +777,15 @@ public class Balancer {
LOG.warn("Exception while getting block list", e);
return;
}
}
} else {
// source node cannot find a pendingBlockToMove, iteration +1
noPendingBlockIteration++;
// in case no blocks can be moved for source node's task,
// jump out of while-loop after 5 iterations.
if (noPendingBlockIteration >= MAX_NO_PENDING_BLOCK_ITERATIONS) {
scheduledSize = 0;
}
}
// check if time is up or not
if (Time.now()-startTime > MAX_ITERATION_TIME) {
@ -801,8 +811,8 @@ public class Balancer {
*/
private static void checkReplicationPolicyCompatibility(Configuration conf
) throws UnsupportedActionException {
if (BlockPlacementPolicy.getInstance(conf, null, null) instanceof
BlockPlacementPolicyDefault) {
if (!(BlockPlacementPolicy.getInstance(conf, null, null) instanceof
BlockPlacementPolicyDefault)) {
throw new UnsupportedActionException(
"Balancer without BlockPlacementPolicyDefault");
}
@ -1085,7 +1095,6 @@ public class Balancer {
}
};
private BytesMoved bytesMoved = new BytesMoved();
private int notChangedIterations = 0;
/* Start a thread to dispatch block moves for each source.
* The thread selects blocks to move & sends request to proxy source to
@ -1370,7 +1379,7 @@ public class Balancer {
" in this iteration");
}
formatter.format("%-24s %10d %19s %18s %17s\n",
formatter.format("%-24s %10d %19s %18s %17s%n",
DateFormat.getDateTimeInstance().format(new Date()),
iteration,
StringUtils.byteDesc(bytesMoved.get()),
@ -1384,19 +1393,10 @@ public class Balancer {
* available to move.
* Exit no byte has been moved for 5 consecutive iterations.
*/
if (dispatchBlockMoves() > 0) {
notChangedIterations = 0;
} else {
notChangedIterations++;
if (notChangedIterations >= 5) {
System.out.println(
"No block has been moved for 5 iterations. Exiting...");
return ReturnStatus.NO_MOVE_PROGRESS;
}
if (!this.nnc.shouldContinue(dispatchBlockMoves())) {
return ReturnStatus.NO_MOVE_PROGRESS;
}
// clean all lists
resetData(conf);
return ReturnStatus.IN_PROGRESS;
} catch (IllegalArgumentException e) {
System.out.println(e + ". Exiting ...");
@ -1445,6 +1445,8 @@ public class Balancer {
for(NameNodeConnector nnc : connectors) {
final Balancer b = new Balancer(nnc, p, conf);
final ReturnStatus r = b.run(iteration, formatter, conf);
// clean all lists
b.resetData(conf);
if (r == ReturnStatus.IN_PROGRESS) {
done = false;
} else if (r != ReturnStatus.SUCCESS) {

View File

@ -52,6 +52,7 @@ import org.apache.hadoop.util.Daemon;
class NameNodeConnector {
private static final Log LOG = Balancer.LOG;
private static final Path BALANCER_ID_PATH = new Path("/system/balancer.id");
private static final int MAX_NOT_CHANGED_ITERATIONS = 5;
final URI nameNodeUri;
final String blockpoolID;
@ -65,6 +66,8 @@ class NameNodeConnector {
private final boolean encryptDataTransfer;
private boolean shouldRun;
private long keyUpdaterInterval;
// used for balancer
private int notChangedIterations = 0;
private BlockTokenSecretManager blockTokenSecretManager;
private Daemon keyupdaterthread; // AccessKeyUpdater thread
private DataEncryptionKey encryptionKey;
@ -119,6 +122,20 @@ class NameNodeConnector {
}
}
boolean shouldContinue(long dispatchBlockMoveBytes) {
if (dispatchBlockMoveBytes > 0) {
notChangedIterations = 0;
} else {
notChangedIterations++;
if (notChangedIterations >= MAX_NOT_CHANGED_ITERATIONS) {
System.out.println("No block has been moved for "
+ notChangedIterations + " iterations. Exiting...");
return false;
}
}
return true;
}
/** Get an access token for a block. */
Token<BlockTokenIdentifier> getAccessToken(ExtendedBlock eb
) throws IOException {

View File

@ -33,6 +33,7 @@ import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -107,8 +108,8 @@ public class BlockManager {
private volatile long corruptReplicaBlocksCount = 0L;
private volatile long underReplicatedBlocksCount = 0L;
private volatile long scheduledReplicationBlocksCount = 0L;
private volatile long excessBlocksCount = 0L;
private volatile long postponedMisreplicatedBlocksCount = 0L;
private AtomicLong excessBlocksCount = new AtomicLong(0L);
private AtomicLong postponedMisreplicatedBlocksCount = new AtomicLong(0L);
/** Used by metrics */
public long getPendingReplicationBlocksCount() {
@ -132,11 +133,11 @@ public class BlockManager {
}
/** Used by metrics */
public long getExcessBlocksCount() {
return excessBlocksCount;
return excessBlocksCount.get();
}
/** Used by metrics */
public long getPostponedMisreplicatedBlocksCount() {
return postponedMisreplicatedBlocksCount;
return postponedMisreplicatedBlocksCount.get();
}
/** Used by metrics */
public int getPendingDataNodeMessageCount() {
@ -170,29 +171,34 @@ public class BlockManager {
*/
private final Set<Block> postponedMisreplicatedBlocks = Sets.newHashSet();
//
// Keeps a TreeSet for every named node. Each treeset contains
// a list of the blocks that are "extra" at that location. We'll
// eventually remove these extras.
// Mapping: StorageID -> TreeSet<Block>
//
/**
* Maps a StorageID to the set of blocks that are "extra" for this
* DataNode. We'll eventually remove these extras.
*/
public final Map<String, LightWeightLinkedSet<Block>> excessReplicateMap =
new TreeMap<String, LightWeightLinkedSet<Block>>();
//
// Store set of Blocks that need to be replicated 1 or more times.
// We also store pending replication-orders.
//
/**
* Store set of Blocks that need to be replicated 1 or more times.
* We also store pending replication-orders.
*/
public final UnderReplicatedBlocks neededReplications = new UnderReplicatedBlocks();
@VisibleForTesting
final PendingReplicationBlocks pendingReplications;
/** The maximum number of replicas allowed for a block */
public final short maxReplication;
/** The maximum number of outgoing replication streams
* a given node should have at one time
*/
/**
* The maximum number of outgoing replication streams a given node should have
* at one time considering all but the highest priority replications needed.
*/
int maxReplicationStreams;
/**
* The maximum number of outgoing replication streams a given node should have
* at one time.
*/
int replicationStreamsHardLimit;
/** Minimum copies needed or else write is disallowed */
public final short minReplication;
/** Default number of replicas */
@ -263,9 +269,16 @@ public class BlockManager {
this.minReplication = (short)minR;
this.maxReplication = (short)maxR;
this.maxReplicationStreams = conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT);
this.shouldCheckForEnoughRacks = conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) != null;
this.maxReplicationStreams =
conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT);
this.replicationStreamsHardLimit =
conf.getInt(
DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_DEFAULT);
this.shouldCheckForEnoughRacks =
conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) == null
? false : true;
this.blocksInvalidateWorkPct = DFSUtil.getInvalidateWorkPctPerIteration(conf);
this.blocksReplWorkMultiplier = DFSUtil.getReplWorkMultiplier(conf);
@ -435,7 +448,8 @@ public class BlockManager {
NumberReplicas numReplicas = new NumberReplicas();
// source node returned is not used
chooseSourceDatanode(block, containingNodes,
containingLiveReplicasNodes, numReplicas);
containingLiveReplicasNodes, numReplicas,
UnderReplicatedBlocks.LEVEL);
assert containingLiveReplicasNodes.size() == numReplicas.liveReplicas();
int usableReplicas = numReplicas.liveReplicas() +
numReplicas.decommissionedReplicas();
@ -1052,7 +1066,7 @@ public class BlockManager {
private void postponeBlock(Block blk) {
if (postponedMisreplicatedBlocks.add(blk)) {
postponedMisreplicatedBlocksCount++;
postponedMisreplicatedBlocksCount.incrementAndGet();
}
}
@ -1145,11 +1159,12 @@ public class BlockManager {
liveReplicaNodes = new ArrayList<DatanodeDescriptor>();
NumberReplicas numReplicas = new NumberReplicas();
srcNode = chooseSourceDatanode(
block, containingNodes, liveReplicaNodes, numReplicas);
block, containingNodes, liveReplicaNodes, numReplicas,
priority);
if(srcNode == null) { // block can not be replicated from any node
LOG.debug("Block " + block + " cannot be repl from any node");
continue;
}
}
assert liveReplicaNodes.size() == numReplicas.liveReplicas();
// do not schedule more if enough replicas is already pending
@ -1339,16 +1354,34 @@ public class BlockManager {
* since the former do not have write traffic and hence are less busy.
* We do not use already decommissioned nodes as a source.
* Otherwise we choose a random node among those that did not reach their
* replication limit.
* replication limits. However, if the replication is of the highest priority
* and all nodes have reached their replication limits, we will choose a
* random node despite the replication limit.
*
* In addition form a list of all nodes containing the block
* and calculate its replication numbers.
*
* @param block Block for which a replication source is needed
* @param containingNodes List to be populated with nodes found to contain the
* given block
* @param nodesContainingLiveReplicas List to be populated with nodes found to
* contain live replicas of the given block
* @param numReplicas NumberReplicas instance to be initialized with the
* counts of live, corrupt, excess, and
* decommissioned replicas of the given
* block.
* @param priority integer representing replication priority of the given
* block
* @return the DatanodeDescriptor of the chosen node from which to replicate
* the given block
*/
private DatanodeDescriptor chooseSourceDatanode(
@VisibleForTesting
DatanodeDescriptor chooseSourceDatanode(
Block block,
List<DatanodeDescriptor> containingNodes,
List<DatanodeDescriptor> nodesContainingLiveReplicas,
NumberReplicas numReplicas) {
NumberReplicas numReplicas,
int priority) {
containingNodes.clear();
nodesContainingLiveReplicas.clear();
DatanodeDescriptor srcNode = null;
@ -1377,8 +1410,15 @@ public class BlockManager {
// If so, do not select the node as src node
if ((nodesCorrupt != null) && nodesCorrupt.contains(node))
continue;
if(node.getNumberOfBlocksToBeReplicated() >= maxReplicationStreams)
if(priority != UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY
&& node.getNumberOfBlocksToBeReplicated() >= maxReplicationStreams)
{
continue; // already reached replication limit
}
if (node.getNumberOfBlocksToBeReplicated() >= replicationStreamsHardLimit)
{
continue;
}
// the block must not be scheduled for removal on srcNode
if(excessBlocks != null && excessBlocks.contains(block))
continue;
@ -1558,7 +1598,7 @@ public class BlockManager {
"in block map.");
}
it.remove();
postponedMisreplicatedBlocksCount--;
postponedMisreplicatedBlocksCount.decrementAndGet();
continue;
}
MisReplicationResult res = processMisReplicatedBlock(bi);
@ -1568,7 +1608,7 @@ public class BlockManager {
}
if (res != MisReplicationResult.POSTPONE) {
it.remove();
postponedMisreplicatedBlocksCount--;
postponedMisreplicatedBlocksCount.decrementAndGet();
}
}
}
@ -2405,7 +2445,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
excessReplicateMap.put(dn.getStorageID(), excessBlocks);
}
if (excessBlocks.add(block)) {
excessBlocksCount++;
excessBlocksCount.incrementAndGet();
if(blockLog.isDebugEnabled()) {
blockLog.debug("BLOCK* addToExcessReplicate:"
+ " (" + dn + ", " + block
@ -2453,7 +2493,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
.getStorageID());
if (excessBlocks != null) {
if (excessBlocks.remove(block)) {
excessBlocksCount--;
excessBlocksCount.decrementAndGet();
if(blockLog.isDebugEnabled()) {
blockLog.debug("BLOCK* removeStoredBlock: "
+ block + " is removed from excessBlocks");
@ -2798,7 +2838,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
// Remove the block from pendingReplications
pendingReplications.remove(block);
if (postponedMisreplicatedBlocks.remove(block)) {
postponedMisreplicatedBlocksCount--;
postponedMisreplicatedBlocksCount.decrementAndGet();
}
}

View File

@ -236,13 +236,18 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
+ totalReplicasExpected + "\n"
+ e.getMessage());
if (avoidStaleNodes) {
// excludedNodes now has - initial excludedNodes, any nodes that were
// chosen and nodes that were tried but were not chosen because they
// were stale, decommissioned or for any other reason a node is not
// chosen for write. Retry again now not avoiding stale node
// Retry chooseTarget again, this time not avoiding stale nodes.
// excludedNodes contains the initial excludedNodes and nodes that were
// not chosen because they were stale, decommissioned, etc.
// We need to additionally exclude the nodes that were added to the
// result list in the successful calls to choose*() above.
for (Node node : results) {
oldExcludedNodes.put(node, node);
}
// Set numOfReplicas, since it can get out of sync with the result list
// if the NotEnoughReplicasException was thrown in chooseRandom().
numOfReplicas = totalReplicasExpected - results.size();
return chooseTarget(numOfReplicas, writer, oldExcludedNodes, blocksize,
maxNodesPerRack, results, false);
}
@ -542,7 +547,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
if (LOG.isDebugEnabled()) {
threadLocalBuilder.get().append(node.toString()).append(": ")
.append("Node ").append(NodeBase.getPath(node))
.append(" is not chosen because the node is staled ");
.append(" is not chosen because the node is stale ");
}
return false;
}

View File

@ -17,19 +17,18 @@
*/
package org.apache.hadoop.hdfs.server.common;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.util.SequentialNumber;
/****************************************************************
* A GenerationStamp is a Hadoop FS primitive, identified by a long.
****************************************************************/
@InterfaceAudience.Private
public class GenerationStamp implements Comparable<GenerationStamp> {
public class GenerationStamp extends SequentialNumber {
/**
* The first valid generation stamp.
* The last reserved generation stamp.
*/
public static final long FIRST_VALID_STAMP = 1000L;
public static final long LAST_RESERVED_STAMP = 1000L;
/**
* Generation stamp of blocks that pre-date the introduction
@ -37,62 +36,10 @@ public class GenerationStamp implements Comparable<GenerationStamp> {
*/
public static final long GRANDFATHER_GENERATION_STAMP = 0;
private AtomicLong genstamp = new AtomicLong();
/**
* Create a new instance, initialized to FIRST_VALID_STAMP.
* Create a new instance, initialized to {@link #LAST_RESERVED_STAMP}.
*/
public GenerationStamp() {
this(GenerationStamp.FIRST_VALID_STAMP);
}
/**
* Create a new instance, initialized to the specified value.
*/
GenerationStamp(long stamp) {
genstamp.set(stamp);
}
/**
* Returns the current generation stamp
*/
public long getStamp() {
return genstamp.get();
}
/**
* Sets the current generation stamp
*/
public void setStamp(long stamp) {
genstamp.set(stamp);
}
/**
* First increments the counter and then returns the stamp
*/
public long nextStamp() {
return genstamp.incrementAndGet();
}
@Override // Comparable
public int compareTo(GenerationStamp that) {
long stamp1 = this.genstamp.get();
long stamp2 = that.genstamp.get();
return stamp1 < stamp2 ? -1 :
stamp1 > stamp2 ? 1 : 0;
}
@Override // Object
public boolean equals(Object o) {
if (!(o instanceof GenerationStamp)) {
return false;
}
return compareTo((GenerationStamp)o) == 0;
}
@Override // Object
public int hashCode() {
long stamp = genstamp.get();
return (int) (stamp^(stamp>>>32));
super(LAST_RESERVED_STAMP);
}
}

View File

@ -59,6 +59,7 @@ import org.apache.hadoop.hdfs.web.resources.DelegationParam;
import org.apache.hadoop.hdfs.web.resources.DoAsParam;
import org.apache.hadoop.hdfs.web.resources.UserParam;
import org.apache.hadoop.http.HtmlQuoting;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.SecurityUtil;
@ -69,6 +70,8 @@ import org.apache.hadoop.security.authorize.ProxyUsers;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.VersionInfo;
import com.google.common.base.Charsets;
import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER;
import static org.apache.hadoop.fs.CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER;
@ -178,7 +181,7 @@ public class JspHelper {
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
} catch (IOException e) {
deadNodes.add(chosenNode);
s.close();
IOUtils.closeSocket(s);
s = null;
failures++;
}
@ -228,7 +231,7 @@ public class JspHelper {
}
blockReader = null;
s.close();
out.print(HtmlQuoting.quoteHtmlChars(new String(buf)));
out.print(HtmlQuoting.quoteHtmlChars(new String(buf, Charsets.UTF_8)));
}
public static void addTableHeader(JspWriter out) throws IOException {
@ -382,6 +385,8 @@ public class JspHelper {
int dint = d1.getVolumeFailures() - d2.getVolumeFailures();
ret = (dint < 0) ? -1 : ((dint > 0) ? 1 : 0);
break;
default:
throw new IllegalArgumentException("Invalid sortField");
}
return (sortOrder == SORT_ORDER_DSC) ? -ret : ret;
}

View File

@ -44,6 +44,8 @@ import org.apache.hadoop.util.VersionInfo;
import com.google.common.base.Preconditions;
import com.google.common.base.Charsets;
/**
@ -658,7 +660,7 @@ public abstract class Storage extends StorageInfo {
FileLock res = null;
try {
res = file.getChannel().tryLock();
file.write(jvmName.getBytes());
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()

View File

@ -703,7 +703,7 @@ class BlockPoolSliceScanner {
(info.lastScanType == ScanType.VERIFICATION_SCAN) ? "local" : "none";
buffer.append(String.format("%-26s : status : %-6s type : %-6s" +
" scan time : " +
"%-15d %s\n", info.block,
"%-15d %s%n", info.block,
(info.lastScanOk ? "ok" : "failed"),
scanType, scanTime,
(scanTime <= 0) ? "not yet verified" :
@ -716,21 +716,21 @@ class BlockPoolSliceScanner {
double pctProgress = (totalBytesToScan == 0) ? 100 :
(totalBytesToScan-bytesLeft)*100.0/totalBytesToScan;
buffer.append(String.format("\nTotal Blocks : %6d" +
"\nVerified in last hour : %6d" +
"\nVerified in last day : %6d" +
"\nVerified in last week : %6d" +
"\nVerified in last four weeks : %6d" +
"\nVerified in SCAN_PERIOD : %6d" +
"\nNot yet verified : %6d" +
"\nVerified since restart : %6d" +
"\nScans since restart : %6d" +
"\nScan errors since restart : %6d" +
"\nTransient scan errors : %6d" +
"\nCurrent scan rate limit KBps : %6d" +
"\nProgress this period : %6.0f%%" +
"\nTime left in cur period : %6.2f%%" +
"\n",
buffer.append(String.format("%nTotal Blocks : %6d" +
"%nVerified in last hour : %6d" +
"%nVerified in last day : %6d" +
"%nVerified in last week : %6d" +
"%nVerified in last four weeks : %6d" +
"%nVerified in SCAN_PERIOD : %6d" +
"%nNot yet verified : %6d" +
"%nVerified since restart : %6d" +
"%nScans since restart : %6d" +
"%nScan errors since restart : %6d" +
"%nTransient scan errors : %6d" +
"%nCurrent scan rate limit KBps : %6d" +
"%nProgress this period : %6.0f%%" +
"%nTime left in cur period : %6.2f%%" +
"%n",
total, inOneHour, inOneDay, inOneWeek,
inFourWeeks, inScanPeriod, neverScanned,
totalScans, totalScans,

View File

@ -78,6 +78,10 @@ public class BlockPoolSliceStorage extends Storage {
this.clusterID = clusterId;
}
private BlockPoolSliceStorage() {
super(NodeType.DATA_NODE);
}
/**
* Analyze storage directories. Recover from previous transitions if required.
*
@ -378,7 +382,7 @@ public class BlockPoolSliceStorage extends Storage {
if (!prevDir.exists())
return;
// read attributes out of the VERSION file of previous directory
DataStorage prevInfo = new DataStorage();
BlockPoolSliceStorage prevInfo = new BlockPoolSliceStorage();
prevInfo.readPreviousVersionProperties(bpSd);
// We allow rollback to a state, which is either consistent with

View File

@ -648,7 +648,7 @@ class BlockSender implements java.io.Closeable {
ByteBuffer pktBuf = ByteBuffer.allocate(pktBufSize);
while (endOffset > offset) {
while (endOffset > offset && !Thread.currentThread().isInterrupted()) {
manageOsCache();
long len = sendPacket(pktBuf, maxChunksPerPacket, streamForSendChunks,
transferTo, throttler);
@ -656,16 +656,19 @@ class BlockSender implements java.io.Closeable {
totalRead += len + (numberOfChunks(len) * checksumSize);
seqno++;
}
try {
// send an empty packet to mark the end of the block
sendPacket(pktBuf, maxChunksPerPacket, streamForSendChunks, transferTo,
throttler);
out.flush();
} catch (IOException e) { //socket error
throw ioeToSocketException(e);
}
// If this thread was interrupted, then it did not send the full block.
if (!Thread.currentThread().isInterrupted()) {
try {
// send an empty packet to mark the end of the block
sendPacket(pktBuf, maxChunksPerPacket, streamForSendChunks, transferTo,
throttler);
out.flush();
} catch (IOException e) { //socket error
throw ioeToSocketException(e);
}
sentEntireByteRange = true;
sentEntireByteRange = true;
}
} finally {
if (clientTraceFmt != null) {
final long endTime = System.nanoTime();

View File

@ -98,7 +98,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
@ -115,6 +114,7 @@ import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolServerSideTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
@ -970,29 +970,27 @@ public class DataNode extends Configured
dnId.setStorageID(createNewStorageId(dnId.getXferPort()));
}
/**
* @return a unique storage ID of form "DS-randInt-ipaddr-port-timestamp"
*/
static String createNewStorageId(int port) {
/* Return
* "DS-randInt-ipaddr-currentTimeMillis"
* It is considered extermely rare for all these numbers to match
* on a different machine accidentally for the following
* a) SecureRandom(INT_MAX) is pretty much random (1 in 2 billion), and
* b) Good chance ip address would be different, and
* c) Even on the same machine, Datanode is designed to use different ports.
* d) Good chance that these are started at different times.
* For a confict to occur all the 4 above have to match!.
* The format of this string can be changed anytime in future without
* affecting its functionality.
*/
// It is unlikely that we will create a non-unique storage ID
// for the following reasons:
// a) SecureRandom is a cryptographically strong random number generator
// b) IP addresses will likely differ on different hosts
// c) DataNode xfer ports will differ on the same host
// d) StorageIDs will likely be generated at different times (in ms)
// A conflict requires that all four conditions are violated.
// NB: The format of this string can be changed in the future without
// requiring that old SotrageIDs be updated.
String ip = "unknownIP";
try {
ip = DNS.getDefaultIP("default");
} catch (UnknownHostException ignored) {
LOG.warn("Could not find ip address of \"default\" inteface.");
LOG.warn("Could not find an IP address for the \"default\" inteface.");
}
int rand = DFSUtil.getSecureRandom().nextInt(Integer.MAX_VALUE);
return "DS-" + rand + "-" + ip + "-" + port + "-"
+ Time.now();
return "DS-" + rand + "-" + ip + "-" + port + "-" + Time.now();
}
/** Ensure the authentication method is kerberos */
@ -1468,7 +1466,7 @@ public class DataNode extends Configured
// read ack
if (isClient) {
DNTransferAckProto closeAck = DNTransferAckProto.parseFrom(
HdfsProtoUtil.vintPrefixed(in));
PBHelper.vintPrefixed(in));
if (LOG.isDebugEnabled()) {
LOG.debug(getClass().getSimpleName() + ": close-ack=" + closeAck);
}
@ -1908,10 +1906,11 @@ public class DataNode extends Configured
}
/**
* Get namenode corresponding to a block pool
* Get the NameNode corresponding to the given block pool.
*
* @param bpid Block pool Id
* @return Namenode corresponding to the bpid
* @throws IOException
* @throws IOException if unable to get the corresponding NameNode
*/
public DatanodeProtocolClientSideTranslatorPB getActiveNamenodeForBP(String bpid)
throws IOException {
@ -1935,11 +1934,6 @@ public class DataNode extends Configured
final String bpid = block.getBlockPoolId();
DatanodeProtocolClientSideTranslatorPB nn =
getActiveNamenodeForBP(block.getBlockPoolId());
if (nn == null) {
throw new IOException(
"Unable to synchronize block " + rBlock + ", since this DN "
+ " has not acknowledged any NN as active.");
}
long recoveryId = rBlock.getNewGenerationStamp();
if (LOG.isDebugEnabled()) {

View File

@ -62,7 +62,7 @@ import org.apache.hadoop.util.DiskChecker;
*/
@InterfaceAudience.Private
public class DataStorage extends Storage {
// Constants
public final static String BLOCK_SUBDIR_PREFIX = "subdir";
final static String BLOCK_FILE_PREFIX = "blk_";
final static String COPY_FILE_PREFIX = "dncp_";
@ -71,13 +71,13 @@ public class DataStorage extends Storage {
public final static String STORAGE_DIR_FINALIZED = "finalized";
public final static String STORAGE_DIR_TMP = "tmp";
/** Access to this variable is guarded by "this" */
/** Unique storage ID. {@see DataNode#createNewStorageId(int)} for details */
private String storageID;
// flag to ensure initialzing storage occurs only once
private boolean initilized = false;
// Flag to ensure we only initialize storage once
private boolean initialized = false;
// BlockPoolStorage is map of <Block pool Id, BlockPoolStorage>
// Maps block pool IDs to block pool storage
private Map<String, BlockPoolSliceStorage> bpStorageMap
= Collections.synchronizedMap(new HashMap<String, BlockPoolSliceStorage>());
@ -130,7 +130,7 @@ public class DataStorage extends Storage {
synchronized void recoverTransitionRead(DataNode datanode,
NamespaceInfo nsInfo, Collection<File> dataDirs, StartupOption startOpt)
throws IOException {
if (initilized) {
if (initialized) {
// DN storage has been initialized, no need to do anything
return;
}
@ -200,7 +200,7 @@ public class DataStorage extends Storage {
this.writeAll();
// 4. mark DN storage is initilized
this.initilized = true;
this.initialized = true;
}
/**

View File

@ -42,7 +42,6 @@ import org.apache.commons.logging.Log;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor.InvalidMagicNumberException;
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
@ -56,6 +55,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatus
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@ -144,7 +144,7 @@ class DataXceiver extends Receiver implements Runnable {
/** Return the datanode object. */
DataNode getDataNode() {return datanode;}
private OutputStream getOutputStream() throws IOException {
private OutputStream getOutputStream() {
return socketOut;
}
@ -284,7 +284,7 @@ class DataXceiver extends Receiver implements Runnable {
// to respond with a Status enum.
try {
ClientReadStatusProto stat = ClientReadStatusProto.parseFrom(
HdfsProtoUtil.vintPrefixed(in));
PBHelper.vintPrefixed(in));
if (!stat.hasStatus()) {
LOG.warn("Client " + s.getInetAddress() + " did not send a valid status " +
"code after reading. Will close connection.");
@ -445,7 +445,7 @@ class DataXceiver extends Receiver implements Runnable {
// read connect ack (only for clients, not for replication req)
if (isClient) {
BlockOpResponseProto connectAck =
BlockOpResponseProto.parseFrom(HdfsProtoUtil.vintPrefixed(mirrorIn));
BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(mirrorIn));
mirrorInStatus = connectAck.getStatus();
firstBadLink = connectAck.getFirstBadLink();
if (LOG.isDebugEnabled() || mirrorInStatus != SUCCESS) {
@ -606,7 +606,7 @@ class DataXceiver extends Receiver implements Runnable {
.setBytesPerCrc(bytesPerCRC)
.setCrcPerBlock(crcPerBlock)
.setMd5(ByteString.copyFrom(md5.getDigest()))
.setCrcType(HdfsProtoUtil.toProto(checksum.getChecksumType()))
.setCrcType(PBHelper.convert(checksum.getChecksumType()))
)
.build()
.writeDelimitedTo(out);
@ -765,7 +765,7 @@ class DataXceiver extends Receiver implements Runnable {
// receive the response from the proxy
BlockOpResponseProto copyResponse = BlockOpResponseProto.parseFrom(
HdfsProtoUtil.vintPrefixed(proxyReply));
PBHelper.vintPrefixed(proxyReply));
if (copyResponse.getStatus() != SUCCESS) {
if (copyResponse.getStatus() == ERROR_ACCESS_TOKEN) {

View File

@ -79,9 +79,6 @@ public class DatanodeJspHelper {
.getCanonicalHostName();
}
private static final SimpleDateFormat lsDateFormat =
new SimpleDateFormat("yyyy-MM-dd HH:mm");
/**
* Get the default chunk size.
* @param conf the configuration
@ -205,8 +202,8 @@ public class DatanodeJspHelper {
+ JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr);
cols[0] = "<a href=\"" + datanodeUrl + "\">"
+ HtmlQuoting.quoteHtmlChars(localFileName) + "</a>";
cols[5] = lsDateFormat.format(new Date((files[i]
.getModificationTime())));
cols[5] = new SimpleDateFormat("yyyy-MM-dd HH:mm").format(
new Date((files[i].getModificationTime())));
cols[6] = files[i].getPermission().toString();
cols[7] = files[i].getOwner();
cols[8] = files[i].getGroup();

View File

@ -203,9 +203,6 @@ abstract public class ReplicaInfo extends Block implements Replica {
throw new IOException("detachBlock:Block not found. " + this);
}
File meta = getMetaFile();
if (meta == null) {
throw new IOException("Meta file not found for block " + this);
}
if (HardLink.getLinkCount(file) > numLinks) {
DataNode.LOG.info("CopyOnWrite for block " + this);

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.hdfs.server.datanode.fsdataset;
import java.io.Closeable;
import java.io.FileDescriptor;
import java.io.FileInputStream;
import java.io.InputStream;
import org.apache.hadoop.io.IOUtils;
@ -30,9 +32,9 @@ public class ReplicaInputStreams implements Closeable {
private final InputStream checksumIn;
/** Create an object with a data input stream and a checksum input stream. */
public ReplicaInputStreams(InputStream dataIn, InputStream checksumIn) {
this.dataIn = dataIn;
this.checksumIn = checksumIn;
public ReplicaInputStreams(FileDescriptor dataFd, FileDescriptor checksumFd) {
this.dataIn = new FileInputStream(dataFd);
this.checksumIn = new FileInputStream(checksumFd);
}
/** @return the data input stream. */

View File

@ -390,8 +390,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
if (ckoff > 0) {
metaInFile.seek(ckoff);
}
return new ReplicaInputStreams(new FileInputStream(blockInFile.getFD()),
new FileInputStream(metaInFile.getFD()));
return new ReplicaInputStreams(blockInFile.getFD(), metaInFile.getFD());
}
static File moveBlockFiles(Block b, File srcfile, File destdir

View File

@ -19,16 +19,20 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import java.io.BufferedReader;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
import java.io.FileReader;
import java.io.IOException;
import java.io.PrintStream;
import java.io.InputStreamReader;
import java.io.OutputStreamWriter;
import java.io.PrintWriter;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
import com.google.common.base.Charsets;
class RollingLogsImpl implements RollingLogs {
private static final String CURR_SUFFIX = ".curr";
private static final String PREV_SUFFIX = ".prev";
@ -40,7 +44,7 @@ class RollingLogsImpl implements RollingLogs {
private final File curr;
private final File prev;
private PrintStream out; //require synchronized access
private PrintWriter out; //require synchronized access
private Appender appender = new Appender() {
@Override
@ -82,7 +86,8 @@ class RollingLogsImpl implements RollingLogs {
RollingLogsImpl(String dir, String filePrefix) throws FileNotFoundException{
curr = new File(dir, filePrefix + CURR_SUFFIX);
prev = new File(dir, filePrefix + PREV_SUFFIX);
out = new PrintStream(new FileOutputStream(curr, true));
out = new PrintWriter(new OutputStreamWriter(new FileOutputStream(
curr, true), Charsets.UTF_8));
}
@Override
@ -108,7 +113,8 @@ class RollingLogsImpl implements RollingLogs {
synchronized(this) {
appender.close();
final boolean renamed = curr.renameTo(prev);
out = new PrintStream(new FileOutputStream(curr, true));
out = new PrintWriter(new OutputStreamWriter(new FileOutputStream(
curr, true), Charsets.UTF_8));
if (!renamed) {
throw new IOException("Failed to rename " + curr + " to " + prev);
}
@ -163,7 +169,8 @@ class RollingLogsImpl implements RollingLogs {
reader = null;
}
reader = new BufferedReader(new FileReader(file));
reader = new BufferedReader(new InputStreamReader(new FileInputStream(
file), Charsets.UTF_8));
return true;
}

View File

@ -48,6 +48,8 @@ import org.codehaus.jackson.map.ObjectMapper;
import org.codehaus.jackson.type.TypeReference;
import org.znerd.xmlenc.XMLOutputter;
import com.google.common.base.Charsets;
/**
* This class generates the data that is needed to be displayed on cluster web
* console.
@ -873,7 +875,7 @@ class ClusterJspHelper {
URLConnection connection = url.openConnection();
BufferedReader in = new BufferedReader(
new InputStreamReader(
connection.getInputStream()));
connection.getInputStream(), Charsets.UTF_8));
String inputLine;
while ((inputLine = in.readLine()) != null) {
out.append(inputLine);

View File

@ -21,6 +21,8 @@ import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.NameNodeProxies;
import org.apache.hadoop.hdfs.server.common.Storage;
@ -41,6 +43,7 @@ import org.apache.hadoop.security.UserGroupInformation;
* int, int, byte[])
*/
class EditLogBackupOutputStream extends EditLogOutputStream {
private static Log LOG = LogFactory.getLog(EditLogFileOutputStream.class);
static int DEFAULT_BUFFER_SIZE = 256;
private final JournalProtocol backupNode; // RPC proxy to backup node
@ -117,6 +120,11 @@ class EditLogBackupOutputStream extends EditLogOutputStream {
protected void flushAndSync(boolean durable) throws IOException {
assert out.getLength() == 0 : "Output buffer is not empty";
if (doubleBuf.isFlushed()) {
LOG.info("Nothing to flush");
return;
}
int numReadyTxns = doubleBuf.countReadyTxns();
long firstTxToFlush = doubleBuf.getFirstReadyTxId();

View File

@ -140,10 +140,8 @@ public class EditLogFileOutputStream extends EditLogOutputStream {
fc.close();
fc = null;
}
if (fp != null) {
fp.close();
fp = null;
}
fp.close();
fp = null;
} finally {
IOUtils.cleanup(FSNamesystem.LOG, fc, fp);
doubleBuf = null;

View File

@ -81,8 +81,9 @@ import com.google.common.base.Preconditions;
public class FSDirectory implements Closeable {
private static INodeDirectoryWithQuota createRoot(FSNamesystem namesystem) {
final INodeDirectoryWithQuota r = new INodeDirectoryWithQuota(
namesystem.allocateNewInodeId(),
INodeDirectory.ROOT_NAME,
namesystem.createFsOwnerPermissions(new FsPermission((short)0755)));
namesystem.createFsOwnerPermissions(new FsPermission((short) 0755)));
final INodeDirectorySnapshottable s = new INodeDirectorySnapshottable(r);
s.setSnapshotQuota(0);
return s;
@ -262,7 +263,9 @@ public class FSDirectory implements Closeable {
if (!mkdirs(parent.toString(), permissions, true, modTime)) {
return null;
}
long id = namesystem.allocateNewInodeId();
INodeFileUnderConstruction newNode = new INodeFileUnderConstruction(
id,
permissions,replication,
preferredBlockSize, modTime, clientName,
clientMachine, clientNode);
@ -284,7 +287,8 @@ public class FSDirectory implements Closeable {
return newNode;
}
INode unprotectedAddFile( String path,
INode unprotectedAddFile( long id,
String path,
PermissionStatus permissions,
short replication,
long modificationTime,
@ -296,13 +300,11 @@ public class FSDirectory implements Closeable {
final INode newNode;
assert hasWriteLock();
if (underConstruction) {
newNode = new INodeFileUnderConstruction(
permissions, replication,
preferredBlockSize, modificationTime, clientName,
clientMachine, null);
newNode = new INodeFileUnderConstruction(id, permissions, replication,
preferredBlockSize, modificationTime, clientName, clientMachine, null);
} else {
newNode = new INodeFile(permissions, BlockInfo.EMPTY_ARRAY, replication,
modificationTime, atime, preferredBlockSize);
newNode = new INodeFile(id, permissions, BlockInfo.EMPTY_ARRAY,
replication, modificationTime, atime, preferredBlockSize);
}
try {
@ -399,19 +401,16 @@ public class FSDirectory implements Closeable {
/**
* Remove a block from the file.
*/
boolean removeBlock(String path, INodeFileUnderConstruction fileNode,
void removeBlock(String path, INodeFileUnderConstruction fileNode,
Block block) throws IOException {
waitForReady();
writeLock();
try {
unprotectedRemoveBlock(path, fileNode, block);
// write modified block locations to log
fsImage.getEditLog().logOpenFile(path, fileNode);
} finally {
writeUnlock();
}
return true;
}
void unprotectedRemoveBlock(String path, INodeFileUnderConstruction fileNode,
@ -1634,8 +1633,9 @@ public class FSDirectory implements Closeable {
// create directories beginning from the first null index
for(; i < inodes.length; i++) {
pathbuilder.append(Path.SEPARATOR + names[i]);
unprotectedMkdir(inodesInPath, i, components[i],
(i < lastInodeIndex) ? parentPermissions : permissions, now);
unprotectedMkdir(namesystem.allocateNewInodeId(), inodesInPath, i,
components[i], (i < lastInodeIndex) ? parentPermissions
: permissions, now);
if (inodes[i] == null) {
return false;
}
@ -1657,7 +1657,7 @@ public class FSDirectory implements Closeable {
return true;
}
INode unprotectedMkdir(String src, PermissionStatus permissions,
INode unprotectedMkdir(long inodeId, String src, PermissionStatus permissions,
long timestamp) throws QuotaExceededException,
UnresolvedLinkException {
assert hasWriteLock();
@ -1666,7 +1666,8 @@ public class FSDirectory implements Closeable {
components.length, false);
INode[] inodes = inodesInPath.getINodes();
final int pos = inodes.length - 1;
unprotectedMkdir(inodesInPath, pos, components[pos], permissions, timestamp);
unprotectedMkdir(inodeId, inodesInPath, pos, components[pos], permissions,
timestamp);
return inodes[pos];
}
@ -1674,11 +1675,12 @@ public class FSDirectory implements Closeable {
* The parent path to the directory is at [0, pos-1].
* All ancestors exist. Newly created one stored at index pos.
*/
private void unprotectedMkdir(INodesInPath inodesInPath, int pos,
byte[] name, PermissionStatus permission,
long timestamp) throws QuotaExceededException {
private void unprotectedMkdir(long inodeId, INodesInPath inodesInPath,
int pos, byte[] name, PermissionStatus permission, long timestamp)
throws QuotaExceededException {
assert hasWriteLock();
final INodeDirectory dir = new INodeDirectory(name, permission, timestamp);
final INodeDirectory dir = new INodeDirectory(inodeId, name, permission,
timestamp);
if (addChild(inodesInPath, pos, dir, true)) {
inodesInPath.setINode(pos, dir);
}
@ -2248,9 +2250,10 @@ public class FSDirectory implements Closeable {
}
final String userName = dirPerms.getUserName();
INodeSymlink newNode = null;
long id = namesystem.allocateNewInodeId();
writeLock();
try {
newNode = unprotectedAddSymlink(path, target, modTime, modTime,
newNode = unprotectedAddSymlink(id, path, target, modTime, modTime,
new PermissionStatus(userName, null, FsPermission.getDefault()));
} finally {
writeUnlock();
@ -2270,12 +2273,13 @@ public class FSDirectory implements Closeable {
/**
* Add the specified path into the namespace. Invoked from edit log processing.
*/
INodeSymlink unprotectedAddSymlink(String path, String target, long mtime,
long atime, PermissionStatus perm)
INodeSymlink unprotectedAddSymlink(long id, String path, String target,
long mtime, long atime, PermissionStatus perm)
throws UnresolvedLinkException, QuotaExceededException {
assert hasWriteLock();
final INodeSymlink symlink = new INodeSymlink(target, mtime, atime, perm);
return addINode(path, symlink)? symlink: null;
final INodeSymlink symlink = new INodeSymlink(id, target, mtime, atime,
perm);
return addINode(path, symlink) ? symlink : null;
}
/**

View File

@ -121,11 +121,8 @@ public class FSEditLogLoader {
long lastTxId = in.getLastTxId();
long numTxns = (lastTxId - expectedStartingTxId) + 1;
long lastLogTime = now();
if (LOG.isDebugEnabled()) {
LOG.debug("edit log length: " + in.length() + ", start txid: "
+ expectedStartingTxId + ", last txid: " + lastTxId);
}
long lastInodeId = fsNamesys.getLastInodeId();
try {
while (true) {
try {
@ -171,7 +168,10 @@ public class FSEditLogLoader {
}
}
try {
applyEditLogOp(op, fsDir, in.getVersion());
long inodeId = applyEditLogOp(op, fsDir, in.getVersion());
if (lastInodeId < inodeId) {
lastInodeId = inodeId;
}
} catch (Throwable e) {
LOG.error("Encountered exception on operation " + op, e);
MetaRecoveryContext.editLogLoaderPrompt("Failed to " +
@ -206,6 +206,7 @@ public class FSEditLogLoader {
}
}
} finally {
fsNamesys.resetLastInodeId(lastInodeId);
if(closeOnExit) {
in.close();
}
@ -224,9 +225,9 @@ public class FSEditLogLoader {
}
@SuppressWarnings("deprecation")
private void applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
int logVersion) throws IOException {
long inodeId = INodeId.GRANDFATHER_INODE_ID;
if (LOG.isTraceEnabled()) {
LOG.trace("replaying edit log: " + op);
}
@ -257,11 +258,11 @@ public class FSEditLogLoader {
assert addCloseOp.blocks.length == 0;
// add to the file tree
newFile = (INodeFile)fsDir.unprotectedAddFile(
addCloseOp.path, addCloseOp.permissions,
replication, addCloseOp.mtime,
addCloseOp.atime, addCloseOp.blockSize,
true, addCloseOp.clientName, addCloseOp.clientMachine);
inodeId = fsNamesys.allocateNewInodeId();
newFile = (INodeFile) fsDir.unprotectedAddFile(inodeId,
addCloseOp.path, addCloseOp.permissions, replication,
addCloseOp.mtime, addCloseOp.atime, addCloseOp.blockSize, true,
addCloseOp.clientName, addCloseOp.clientMachine);
fsNamesys.leaseManager.addLease(addCloseOp.clientName, addCloseOp.path);
} else { // This is OP_ADD on an existing file
@ -374,7 +375,8 @@ public class FSEditLogLoader {
}
case OP_MKDIR: {
MkdirOp mkdirOp = (MkdirOp)op;
fsDir.unprotectedMkdir(mkdirOp.path, mkdirOp.permissions,
inodeId = fsNamesys.allocateNewInodeId();
fsDir.unprotectedMkdir(inodeId, mkdirOp.path, mkdirOp.permissions,
mkdirOp.timestamp);
break;
}
@ -427,9 +429,10 @@ public class FSEditLogLoader {
}
case OP_SYMLINK: {
SymlinkOp symlinkOp = (SymlinkOp)op;
fsDir.unprotectedAddSymlink(symlinkOp.path, symlinkOp.value,
symlinkOp.mtime, symlinkOp.atime,
symlinkOp.permissionStatus);
inodeId = fsNamesys.allocateNewInodeId();
fsDir.unprotectedAddSymlink(inodeId, symlinkOp.path,
symlinkOp.value, symlinkOp.mtime,
symlinkOp.atime, symlinkOp.permissionStatus);
break;
}
case OP_RENAME: {
@ -489,6 +492,7 @@ public class FSEditLogLoader {
default:
throw new IOException("Invalid operation read " + op.opCode);
}
return inodeId;
}
private static String formatEditLogReplayError(EditLogInputStream in,

View File

@ -59,6 +59,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.util.Canceler;
import org.apache.hadoop.hdfs.util.MD5FileUtils;
import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.util.IdGenerator;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
@ -92,6 +93,7 @@ public class FSImage implements Closeable {
final private Configuration conf;
protected NNStorageRetentionManager archivalManager;
protected IdGenerator blockIdGenerator;
/**
* Construct an FSImage
@ -137,6 +139,9 @@ public class FSImage implements Closeable {
Preconditions.checkState(fileCount == 1,
"FSImage.format should be called with an uninitialized namesystem, has " +
fileCount + " files");
// BlockIdGenerator is defined during formatting
// currently there is only one BlockIdGenerator
blockIdGenerator = createBlockIdGenerator(fsn);
NamespaceInfo ns = NNStorage.newNamespaceInfo();
ns.clusterID = clusterId;
@ -253,6 +258,7 @@ public class FSImage implements Closeable {
doRollback();
break;
case REGULAR:
default:
// just load the image
}
@ -737,6 +743,9 @@ public class FSImage implements Closeable {
FSImageFormat.Loader loader = new FSImageFormat.Loader(
conf, target);
loader.load(curFile);
// BlockIdGenerator is determined after loading image
// currently there is only one BlockIdGenerator
blockIdGenerator = createBlockIdGenerator(target);
target.setBlockPoolId(this.getBlockPoolID());
// Check that the image digest we loaded matches up with what
@ -1165,4 +1174,12 @@ public class FSImage implements Closeable {
public synchronized long getMostRecentCheckpointTxId() {
return storage.getMostRecentCheckpointTxId();
}
public long getUniqueBlockId() {
return blockIdGenerator.nextValue();
}
public IdGenerator createBlockIdGenerator(FSNamesystem fsn) {
return new RandomBlockIdGenerator(fsn);
}
}

View File

@ -220,7 +220,8 @@ class FSImageFormat {
in = compression.unwrapInputStream(fin);
LOG.info("Loading image file " + curFile + " using " + compression);
// reset INodeId. TODO: remove this after inodeId is persisted in fsimage
namesystem.resetLastInodeIdWithoutChecking(INodeId.LAST_RESERVED_ID);
// load all inodes
LOG.info("Number of files = " + numFiles);
if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
@ -400,6 +401,8 @@ class FSImageFormat {
long blockSize = 0;
int imgVersion = getLayoutVersion();
long inodeId = namesystem.allocateNewInodeId();
short replication = in.readShort();
replication = namesystem.getBlockManager().adjustReplication(replication);
modificationTime = in.readLong();
@ -437,7 +440,7 @@ class FSImageFormat {
PermissionStatus permissions = PermissionStatus.read(in);
return INode.newINode(permissions, blocks, symlink, replication,
return INode.newINode(inodeId, permissions, blocks, symlink, replication,
modificationTime, atime, nsQuota, dsQuota, blockSize);
}

View File

@ -107,7 +107,9 @@ public class FSImageSerialization {
int numLocs = in.readInt();
assert numLocs == 0 : "Unexpected block locations";
return new INodeFileUnderConstruction(name,
//TODO: get inodeId from fsimage after inodeId is persisted
return new INodeFileUnderConstruction(INodeId.GRANDFATHER_INODE_ID,
name,
blockReplication,
modificationTime,
preferredBlockSize,

View File

@ -78,8 +78,9 @@ import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.FileWriter;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStreamWriter;
import java.io.PrintWriter;
import java.io.StringWriter;
import java.lang.management.ManagementFactory;
@ -211,6 +212,7 @@ import org.apache.hadoop.util.VersionInfo;
import org.mortbay.util.ajax.JSON;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
@ -383,14 +385,43 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
private final boolean haEnabled;
private INodeId inodeId;
/**
* Set the last allocated inode id when fsimage or editlog is loaded.
*/
public void resetLastInodeId(long newValue) throws IOException {
try {
inodeId.skipTo(newValue);
} catch(IllegalStateException ise) {
throw new IOException(ise);
}
}
/** Should only be used for tests to reset to any value */
void resetLastInodeIdWithoutChecking(long newValue) {
inodeId.setCurrentValue(newValue);
}
/** @return the last inode ID. */
public long getLastInodeId() {
return inodeId.getCurrentValue();
}
/** Allocate a new inode ID. */
public long allocateNewInodeId() {
return inodeId.nextValue();
}
/**
* Clear all loaded data
*/
void clear() {
dir.reset();
dtSecretManager.reset();
generationStamp.setStamp(GenerationStamp.FIRST_VALID_STAMP);
generationStamp.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
leaseManager.removeAllLeases();
inodeId.setCurrentValue(INodeId.LAST_RESERVED_ID);
}
@VisibleForTesting
@ -542,6 +573,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
this.standbyShouldCheckpoint = conf.getBoolean(
DFS_HA_STANDBY_CHECKPOINTS_KEY, DFS_HA_STANDBY_CHECKPOINTS_DEFAULT);
this.inodeId = new INodeId();
// For testing purposes, allow the DT secret manager to be started regardless
// of whether security is enabled.
alwaysUseDelegationTokensForTests = conf.getBoolean(
@ -1067,8 +1100,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
try {
checkSuperuserPrivilege();
File file = new File(System.getProperty("hadoop.log.dir"), filename);
PrintWriter out = new PrintWriter(new BufferedWriter(new FileWriter(file,
true)));
PrintWriter out = new PrintWriter(new BufferedWriter(
new OutputStreamWriter(new FileOutputStream(file, true), Charsets.UTF_8)));
metaSave(out);
out.flush();
out.close();
@ -2520,13 +2553,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
private Block allocateBlock(String src, INodesInPath inodesInPath,
DatanodeDescriptor targets[]) throws IOException {
assert hasWriteLock();
Block b = new Block(DFSUtil.getRandom().nextLong(), 0, 0);
while(isValidBlock(b)) {
b.setBlockId(DFSUtil.getRandom().nextLong());
}
Block b = new Block(getFSImage().getUniqueBlockId(), 0, 0);
// Increment the generation stamp for every new block.
nextGenerationStamp();
b.setGenerationStamp(getGenerationStamp());
b.setGenerationStamp(nextGenerationStamp());
b = dir.addBlock(src, inodesInPath, b, targets);
NameNode.stateChangeLog.info("BLOCK* allocateBlock: " + src + ". "
+ blockPoolId + " " + b);
@ -4321,6 +4350,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
case SAFEMODE_ENTER: // enter safe mode
enterSafeMode(false);
break;
default:
LOG.error("Unexpected safe mode action");
}
}
return isInSafeMode();
@ -4580,13 +4611,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
}
/**
* Returns whether the given block is one pointed-to by a file.
*/
private boolean isValidBlock(Block b) {
return (blockManager.getBlockCollection(b) != null);
}
PermissionStatus createFsOwnerPermissions(FsPermission permission) {
return new PermissionStatus(fsOwner.getShortUserName(), supergroup, permission);
}
@ -4796,14 +4820,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
* Sets the generation stamp for this filesystem
*/
void setGenerationStamp(long stamp) {
generationStamp.setStamp(stamp);
generationStamp.setCurrentValue(stamp);
}
/**
* Gets the generation stamp for this filesystem
*/
long getGenerationStamp() {
return generationStamp.getStamp();
return generationStamp.getCurrentValue();
}
/**
@ -4815,7 +4839,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
throw new SafeModeException(
"Cannot get next generation stamp", safeMode);
}
long gs = generationStamp.nextStamp();
final long gs = generationStamp.nextValue();
getEditLog().logGenerationStamp(gs);
// NB: callers sync the log
return gs;

View File

@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
import java.io.PrintWriter;
import java.io.StringWriter;
import java.util.Arrays;
@ -142,6 +143,11 @@ public abstract class INode implements Comparable<byte[]> {
}
}
/**
* The inode id
*/
final private long id;
/**
* The inode name is in java UTF8 encoding;
* The name in HdfsFileStatus should keep the same encoding as this.
@ -161,8 +167,9 @@ public abstract class INode implements Comparable<byte[]> {
private long modificationTime = 0L;
private long accessTime = 0L;
private INode(byte[] name, long permission, INodeDirectory parent,
private INode(long id, byte[] name, long permission, INodeDirectory parent,
long modificationTime, long accessTime) {
this.id = id;
this.name = name;
this.permission = permission;
this.parent = parent;
@ -170,26 +177,31 @@ public abstract class INode implements Comparable<byte[]> {
this.accessTime = accessTime;
}
INode(byte[] name, PermissionStatus permissions, INodeDirectory parent,
long modificationTime, long accessTime) {
this(name, PermissionStatusFormat.toLong(permissions), parent,
INode(long id, byte[] name, PermissionStatus permissions,
INodeDirectory parent, long modificationTime, long accessTime) {
this(id, name, PermissionStatusFormat.toLong(permissions), parent,
modificationTime, accessTime);
}
INode(PermissionStatus permissions, long mtime, long atime) {
this(null, permissions, null, mtime, atime);
INode(long id, PermissionStatus permissions, long mtime, long atime) {
this(id, null, PermissionStatusFormat.toLong(permissions), null, mtime, atime);
}
protected INode(String name, PermissionStatus permissions) {
this(DFSUtil.string2Bytes(name), permissions, null, 0L, 0L);
protected INode(long id, String name, PermissionStatus permissions) {
this(id, DFSUtil.string2Bytes(name), permissions, null, 0L, 0L);
}
/** @param other Other node to be copied */
INode(INode other) {
this(other.name, other.permission, other.parent,
this(other.id, other.name, other.permission, other.parent,
other.modificationTime, other.accessTime);
}
/** Get inode id */
public long getId() {
return this.id;
}
/**
* Create a copy of this inode for snapshot.
*
@ -598,6 +610,7 @@ public abstract class INode implements Comparable<byte[]> {
/**
* Create an INode; the inode's name is not set yet
*
* @param id preassigned inode id
* @param permissions permissions
* @param blocks blocks if a file
* @param symlink symblic link if a symbolic link
@ -609,7 +622,8 @@ public abstract class INode implements Comparable<byte[]> {
* @param preferredBlockSize block size
* @return an inode
*/
static INode newINode(PermissionStatus permissions,
static INode newINode(long id,
PermissionStatus permissions,
BlockInfo[] blocks,
String symlink,
short replication,
@ -619,17 +633,17 @@ public abstract class INode implements Comparable<byte[]> {
long dsQuota,
long preferredBlockSize) {
if (symlink.length() != 0) { // check if symbolic link
return new INodeSymlink(symlink, modificationTime, atime, permissions);
return new INodeSymlink(id, symlink, modificationTime, atime, permissions);
} else if (blocks == null) { //not sym link and blocks null? directory!
if (nsQuota >= 0 || dsQuota >= 0) {
return new INodeDirectoryWithQuota(
permissions, modificationTime, nsQuota, dsQuota);
id, permissions, modificationTime, nsQuota, dsQuota);
}
// regular directory
return new INodeDirectory(permissions, modificationTime);
return new INodeDirectory(id, permissions, modificationTime);
}
// file
return new INodeFile(permissions, blocks, replication,
return new INodeFile(id, permissions, blocks, replication,
modificationTime, atime, preferredBlockSize);
}

View File

@ -62,17 +62,17 @@ public class INodeDirectory extends INode {
private List<INode> children = null;
public INodeDirectory(String name, PermissionStatus permissions) {
super(name, permissions);
public INodeDirectory(long id, String name, PermissionStatus permissions) {
super(id, name, permissions);
}
public INodeDirectory(PermissionStatus permissions, long mTime) {
super(permissions, mTime, 0);
public INodeDirectory(long id, PermissionStatus permissions, long mTime) {
super(id, permissions, mTime, 0);
}
/** constructor */
INodeDirectory(byte[] name, PermissionStatus permissions, long mtime) {
super(name, permissions, null, mtime, 0L);
INodeDirectory(long id, byte[] name, PermissionStatus permissions, long mtime) {
super(id, name, permissions, null, mtime, 0L);
}
/**

View File

@ -55,16 +55,16 @@ public class INodeDirectoryWithQuota extends INodeDirectory {
}
/** constructor with no quota verification */
INodeDirectoryWithQuota(PermissionStatus permissions, long modificationTime,
long nsQuota, long dsQuota) {
super(permissions, modificationTime);
INodeDirectoryWithQuota(long id, PermissionStatus permissions,
long modificationTime, long nsQuota, long dsQuota) {
super(id, permissions, modificationTime);
this.nsQuota = nsQuota;
this.dsQuota = dsQuota;
}
/** constructor with no quota verification */
INodeDirectoryWithQuota(String name, PermissionStatus permissions) {
super(name, permissions);
INodeDirectoryWithQuota(long id, String name, PermissionStatus permissions) {
super(id, name, permissions);
}
/** Get this directory's namespace quota

View File

@ -87,21 +87,21 @@ public class INodeFile extends INode implements BlockCollection {
private BlockInfo[] blocks;
INodeFile(PermissionStatus permissions, BlockInfo[] blklist,
INodeFile(long id, PermissionStatus permissions, BlockInfo[] blklist,
short replication, long modificationTime,
long atime, long preferredBlockSize) {
this(null, permissions, modificationTime, atime, blklist, replication,
this(id, null, permissions, modificationTime, atime, blklist, replication,
preferredBlockSize);
}
protected INodeFile(byte[] name, PermissionStatus permissions, long mtime, long atime,
protected INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime, long atime,
BlockInfo[] blklist, short replication, long preferredBlockSize) {
super(name, permissions, null, mtime, atime);
super(id, name, permissions, null, mtime, atime);
header = HeaderFormat.combineReplication(header, replication);
header = HeaderFormat.combinePreferredBlockSize(header, preferredBlockSize);
this.blocks = blklist;
}
protected INodeFile(INodeFile that) {
super(that);
this.header = that.header;

View File

@ -58,6 +58,7 @@ public class INodeFileUnderConstruction extends INodeFile implements MutableBloc
Preconditions.checkArgument(!(file instanceof INodeFileUnderConstruction),
"file is already an INodeFileUnderConstruction");
final INodeFileUnderConstruction uc = new INodeFileUnderConstruction(
file.getId(),
file.getLocalNameBytes(),
file.getFileReplication(),
file.getModificationTime(),
@ -75,18 +76,20 @@ public class INodeFileUnderConstruction extends INodeFile implements MutableBloc
private final String clientMachine;
private final DatanodeDescriptor clientNode; // if client is a cluster node too.
INodeFileUnderConstruction(PermissionStatus permissions,
INodeFileUnderConstruction(long id,
PermissionStatus permissions,
short replication,
long preferredBlockSize,
long modTime,
String clientName,
String clientMachine,
DatanodeDescriptor clientNode) {
this(null, replication, modTime, preferredBlockSize, BlockInfo.EMPTY_ARRAY,
this(id, null, replication, modTime, preferredBlockSize, BlockInfo.EMPTY_ARRAY,
permissions.applyUMask(UMASK), clientName, clientMachine, clientNode);
}
INodeFileUnderConstruction(byte[] name,
INodeFileUnderConstruction(long id,
byte[] name,
short blockReplication,
long modificationTime,
long preferredBlockSize,
@ -95,7 +98,7 @@ public class INodeFileUnderConstruction extends INodeFile implements MutableBloc
String clientName,
String clientMachine,
DatanodeDescriptor clientNode) {
super(name, perm, modificationTime, modificationTime,
super(id, name, perm, modificationTime, modificationTime,
blocks, blockReplication, preferredBlockSize);
this.clientName = clientName;
this.clientMachine = clientMachine;
@ -140,7 +143,7 @@ public class INodeFileUnderConstruction extends INodeFile implements MutableBloc
*/
protected INodeFile toINodeFile(long mtime) {
assertAllBlocksComplete();
return new INodeFile(getLocalNameBytes(), getPermissionStatus(),
return new INodeFile(getId(), getLocalNameBytes(), getPermissionStatus(),
mtime, getModificationTime(),
getBlocks(), getFileReplication(), getPreferredBlockSize());
}

View File

@ -0,0 +1,44 @@
/**
* 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.server.namenode;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.util.SequentialNumber;
/**
* An id which uniquely identifies an inode
*/
@InterfaceAudience.Private
class INodeId extends SequentialNumber {
/**
* The last reserved inode id. Reserve id 1 to 1000 for potential future
* usage. The id won't be recycled and is not expected to wrap around in a
* very long time. Root inode id will be 1001.
*/
public static final long LAST_RESERVED_ID = 1000L;
/**
* The inode id validation of lease check will be skipped when the request
* uses GRANDFATHER_INODE_ID for backward compatibility.
*/
public static final long GRANDFATHER_INODE_ID = 0;
INodeId() {
super(LAST_RESERVED_ID);
}
}

View File

@ -28,9 +28,9 @@ import org.apache.hadoop.hdfs.DFSUtil;
public class INodeSymlink extends INode {
private final byte[] symlink; // The target URI
INodeSymlink(String value, long mtime, long atime,
PermissionStatus permissions) {
super(permissions, mtime, atime);
INodeSymlink(long id, String value, long mtime, long atime,
PermissionStatus permissions) {
super(id, permissions, mtime, atime);
this.symlink = DFSUtil.string2Bytes(value);
}

View File

@ -164,7 +164,7 @@ public class NameNodeResourceChecker {
CheckedVolume newVolume = new CheckedVolume(dir, required);
CheckedVolume volume = volumes.get(newVolume.getVolume());
if (volume == null || (volume != null && !volume.isRequired())) {
if (volume == null || !volume.isRequired()) {
volumes.put(newVolume.getVolume(), newVolume);
}
}

View File

@ -0,0 +1,52 @@
/**
* 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.server.namenode;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.util.IdGenerator;
/**
* Generator of random block IDs.
*/
@InterfaceAudience.Private
public class RandomBlockIdGenerator implements IdGenerator {
private final BlockManager blockManager;
RandomBlockIdGenerator(FSNamesystem namesystem) {
this.blockManager = namesystem.getBlockManager();
}
@Override // NumberGenerator
public long nextValue() {
Block b = new Block(DFSUtil.getRandom().nextLong(), 0, 0);
while(isValidBlock(b)) {
b.setBlockId(DFSUtil.getRandom().nextLong());
}
return b.getBlockId();
}
/**
* Returns whether the given block is one pointed-to by a file.
*/
private boolean isValidBlock(Block b) {
return (blockManager.getBlockCollection(b) != null);
}
}

View File

@ -17,7 +17,8 @@
package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
import java.io.PrintStream;
import java.io.OutputStreamWriter;
import java.io.PrintWriter;
import java.security.PrivilegedExceptionAction;
import javax.servlet.ServletContext;
@ -32,6 +33,8 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import com.google.common.base.Charsets;
/**
* Renew delegation tokens over http for use in hftp.
*/
@ -73,7 +76,8 @@ public class RenewDelegationTokenServlet extends DfsServlet {
return nn.getRpcServer().renewDelegationToken(token);
}
});
PrintStream os = new PrintStream(resp.getOutputStream());
final PrintWriter os = new PrintWriter(new OutputStreamWriter(
resp.getOutputStream(), Charsets.UTF_8));
os.println(result);
os.close();
} catch(Exception e) {

View File

@ -605,7 +605,9 @@ public class SecondaryNameNode implements Runnable {
terminate(ret);
}
secondary.startCheckpointThread();
if (secondary != null) {
secondary.startCheckpointThread();
}
}

View File

@ -20,7 +20,8 @@ package org.apache.hadoop.hdfs.server.namenode.web.resources;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.OutputStream;
import java.io.PrintStream;
import java.io.OutputStreamWriter;
import java.io.PrintWriter;
import java.net.InetAddress;
import java.net.URI;
import java.net.URISyntaxException;
@ -102,6 +103,7 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import com.google.common.base.Charsets;
import com.sun.jersey.spi.container.ResourceFilters;
/** Web-hdfs NameNode implementation. */
@ -713,7 +715,8 @@ public class NamenodeWebHdfsMethods {
return new StreamingOutput() {
@Override
public void write(final OutputStream outstream) throws IOException {
final PrintStream out = new PrintStream(outstream);
final PrintWriter out = new PrintWriter(new OutputStreamWriter(
outstream, Charsets.UTF_8));
out.println("{\"" + FileStatus.class.getSimpleName() + "es\":{\""
+ FileStatus.class.getSimpleName() + "\":[");
@ -736,6 +739,7 @@ public class NamenodeWebHdfsMethods {
out.println();
out.println("]}}");
out.flush();
}
};
}

View File

@ -90,8 +90,8 @@ public class ReceivedDeletedBlockInfo {
ReceivedDeletedBlockInfo other = (ReceivedDeletedBlockInfo) o;
return this.block.equals(other.getBlock())
&& this.status == other.status
&& (this.delHints == other.delHints ||
this.delHints != null && this.delHints.equals(other.delHints));
&& this.delHints != null
&& this.delHints.equals(other.delHints);
}
@Override

View File

@ -89,6 +89,9 @@ public class RemoteEditLog implements Comparable<RemoteEditLog> {
new Function<RemoteEditLog, Long>() {
@Override
public Long apply(RemoteEditLog log) {
if (null == log) {
return HdfsConstants.INVALID_TXID;
}
return log.getStartTxId();
}
};

View File

@ -55,6 +55,8 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.GenericOptionsParser;
import com.google.common.base.Charsets;
/**
* Fetch a DelegationToken from the current Namenode and store it in the
* specified file.
@ -269,8 +271,8 @@ public class DelegationTokenFetcher {
throw new IOException("Error renewing token: " +
connection.getResponseMessage());
}
in = new BufferedReader(new InputStreamReader
(connection.getInputStream()));
in = new BufferedReader(
new InputStreamReader(connection.getInputStream(), Charsets.UTF_8));
long result = Long.parseLong(in.readLine());
in.close();
return result;

View File

@ -63,7 +63,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
@InterfaceAudience.Private
public class JMXGet {
private static final String format = "%s=%s\n";
private static final String format = "%s=%s%n";
private ArrayList<ObjectName> hadoopObjectNames;
private MBeanServerConnection mbsc;
private String service = "NameNode", port = "", server = "localhost";
@ -126,7 +126,8 @@ public class JMXGet {
continue;
}
}
err("Info: key = " + key + "; val = "+ val.getClass() +":"+ val);
err("Info: key = " + key + "; val = " +
(val == null ? "null" : val.getClass()) + ":" + val);
break;
}

View File

@ -18,9 +18,10 @@
package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.FileReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.util.Stack;
import org.apache.hadoop.classification.InterfaceAudience;
@ -39,6 +40,8 @@ import org.xml.sax.XMLReader;
import org.xml.sax.helpers.DefaultHandler;
import org.xml.sax.helpers.XMLReaderFactory;
import com.google.common.base.Charsets;
/**
* OfflineEditsXmlLoader walks an EditsVisitor over an OEV XML file
*/
@ -48,7 +51,7 @@ class OfflineEditsXmlLoader
extends DefaultHandler implements OfflineEditsLoader {
private final boolean fixTxIds;
private final OfflineEditsVisitor visitor;
private final FileReader fileReader;
private final InputStreamReader fileReader;
private ParseState state;
private Stanza stanza;
private Stack<Stanza> stanzaStack;
@ -70,7 +73,8 @@ class OfflineEditsXmlLoader
public OfflineEditsXmlLoader(OfflineEditsVisitor visitor,
File inputFile, OfflineEditsViewer.Flags flags) throws FileNotFoundException {
this.visitor = visitor;
this.fileReader = new FileReader(inputFile);
this.fileReader =
new InputStreamReader(new FileInputStream(inputFile), Charsets.UTF_8);
this.fixTxIds = flags.getFixTxIds();
}

View File

@ -19,7 +19,8 @@ package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
import java.io.IOException;
import java.io.OutputStream;
import java.io.PrintStream;
import java.io.OutputStreamWriter;
import java.io.PrintWriter;
import java.util.Map;
import java.util.HashMap;
@ -29,6 +30,8 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
import com.google.common.base.Charsets;
/**
* StatisticsEditsVisitor implements text version of EditsVisitor
* that aggregates counts of op codes processed
@ -37,7 +40,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class StatisticsEditsVisitor implements OfflineEditsVisitor {
final private PrintStream out;
final private PrintWriter out;
private int version = -1;
private final Map<FSEditLogOpCodes, Long> opCodeCount =
@ -52,7 +55,7 @@ public class StatisticsEditsVisitor implements OfflineEditsVisitor {
* @param printToScreen Mirror output to screen?
*/
public StatisticsEditsVisitor(OutputStream out) throws IOException {
this.out = new PrintStream(out);
this.out = new PrintWriter(new OutputStreamWriter(out, Charsets.UTF_8));
}
/** Start the visitor */

View File

@ -17,8 +17,12 @@
*/
package org.apache.hadoop.hdfs.tools.offlineImageViewer;
import java.io.FileOutputStream;
import java.io.FileWriter;
import java.io.IOException;
import java.io.OutputStreamWriter;
import com.google.common.base.Charsets;
/**
* TextWriterImageProcessor mixes in the ability for ImageVisitor
@ -34,7 +38,7 @@ import java.io.IOException;
abstract class TextWriterImageVisitor extends ImageVisitor {
private boolean printToScreen = false;
private boolean okToWrite = false;
final private FileWriter fw;
final private OutputStreamWriter fw;
/**
* Create a processor that writes to the file named.
@ -56,7 +60,7 @@ abstract class TextWriterImageVisitor extends ImageVisitor {
throws IOException {
super();
this.printToScreen = printToScreen;
fw = new FileWriter(filename);
fw = new OutputStreamWriter(new FileOutputStream(filename), Charsets.UTF_8);
okToWrite = true;
}

View File

@ -96,7 +96,12 @@ public class DataTransferThrottler {
// Wait for next period so that curReserve can be increased.
try {
wait( curPeriodEnd - now );
} catch (InterruptedException ignored) {}
} catch (InterruptedException e) {
// Abort throttle and reset interrupted status to make sure other
// interrupt handling higher in the call stack executes.
Thread.currentThread().interrupt();
break;
}
} else if ( now < (curPeriodStart + periodExtension)) {
curPeriodStart = curPeriodEnd;
curReserve += bytesPerPeriod;

View File

@ -20,9 +20,9 @@ package org.apache.hadoop.hdfs.util;
import java.io.BufferedReader;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileReader;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.security.DigestInputStream;
import java.security.MessageDigest;
import java.util.regex.Matcher;
@ -34,6 +34,8 @@ import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.util.StringUtils;
import com.google.common.base.Charsets;
/**
* Static functions for dealing with files of the same format
* that the Unix "md5sum" utility writes.
@ -78,7 +80,8 @@ public abstract class MD5FileUtils {
}
BufferedReader reader =
new BufferedReader(new FileReader(md5File));
new BufferedReader(new InputStreamReader(new FileInputStream(
md5File), Charsets.UTF_8));
try {
md5Line = reader.readLine();
if (md5Line == null) { md5Line = ""; }
@ -138,7 +141,7 @@ public abstract class MD5FileUtils {
String md5Line = digestString + " *" + dataFile.getName() + "\n";
AtomicFileOutputStream afos = new AtomicFileOutputStream(md5File);
afos.write(md5Line.getBytes());
afos.write(md5Line.getBytes(Charsets.UTF_8));
afos.close();
LOG.debug("Saved MD5 " + digest + " to " + md5File);
}

View File

@ -19,14 +19,18 @@ package org.apache.hadoop.hdfs.util;
import java.io.BufferedReader;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileReader;
import java.io.IOException;
import java.io.InputStreamReader;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.io.IOUtils;
import com.google.common.base.Charsets;
/**
* Class that represents a file on disk which persistently stores
* a single <code>long</code> value. The file is updated atomically
@ -74,7 +78,7 @@ public class PersistentLongFile {
public static void writeFile(File file, long val) throws IOException {
AtomicFileOutputStream fos = new AtomicFileOutputStream(file);
try {
fos.write(String.valueOf(val).getBytes());
fos.write(String.valueOf(val).getBytes(Charsets.UTF_8));
fos.write('\n');
fos.close();
fos = null;
@ -88,7 +92,9 @@ public class PersistentLongFile {
public static long readFile(File file, long defaultVal) throws IOException {
long val = defaultVal;
if (file.exists()) {
BufferedReader br = new BufferedReader(new FileReader(file));
BufferedReader br =
new BufferedReader(new InputStreamReader(new FileInputStream(
file), Charsets.UTF_8));
try {
val = Long.valueOf(br.readLine());
br.close();

View File

@ -105,6 +105,8 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelect
import org.apache.hadoop.util.Progressable;
import org.mortbay.util.ajax.JSON;
import com.google.common.base.Charsets;
/** A FileSystem for HDFS over the web. */
public class WebHdfsFileSystem extends FileSystem
implements DelegationTokenRenewer.Renewable {
@ -281,7 +283,7 @@ public class WebHdfsFileSystem extends FileSystem
+ "\" (parsed=\"" + parsed + "\")");
}
}
return (Map<?, ?>)JSON.parse(new InputStreamReader(in));
return (Map<?, ?>)JSON.parse(new InputStreamReader(in, Charsets.UTF_8));
}
private static Map<?, ?> validateResponse(final HttpOpParam.Op op,

View File

@ -52,7 +52,7 @@ struct NativeMiniDfsCluster* nmdCreate(struct NativeMiniDfsConf *conf)
if (!env) {
fprintf(stderr, "nmdCreate: unable to construct JNIEnv.\n");
goto error;
return NULL;
}
cl = calloc(1, sizeof(struct NativeMiniDfsCluster));
if (!cl) {

View File

@ -25,6 +25,7 @@ option java_generic_services = true;
option java_generate_equals_and_hash = true;
package hadoop.hdfs;
import "Security.proto";
import "hdfs.proto";
/**
@ -76,7 +77,7 @@ message DeleteBlockPoolResponseProto {
*/
message GetBlockLocalPathInfoRequestProto {
required ExtendedBlockProto block = 1;
required BlockTokenIdentifierProto token = 2;
required hadoop.common.TokenProto token = 2;
}
/**
@ -96,7 +97,7 @@ message GetBlockLocalPathInfoResponseProto {
*/
message GetHdfsBlockLocationsRequestProto {
repeated ExtendedBlockProto blocks = 1;
repeated BlockTokenIdentifierProto tokens = 2;
repeated hadoop.common.TokenProto tokens = 2;
}
/**

View File

@ -22,6 +22,7 @@ option java_generic_services = true;
option java_generate_equals_and_hash = true;
package hadoop.hdfs;
import "Security.proto";
import "hdfs.proto";
/**
@ -419,11 +420,11 @@ message GetDelegationTokenRequestProto {
}
message GetDelegationTokenResponseProto {
required BlockTokenIdentifierProto token = 1;
optional hadoop.common.TokenProto token = 1;
}
message RenewDelegationTokenRequestProto {
required BlockTokenIdentifierProto token = 1;
required hadoop.common.TokenProto token = 1;
}
message RenewDelegationTokenResponseProto {
@ -431,7 +432,7 @@ message RenewDelegationTokenResponseProto {
}
message CancelDelegationTokenRequestProto {
required BlockTokenIdentifierProto token = 1;
required hadoop.common.TokenProto token = 1;
}
message CancelDelegationTokenResponseProto { // void response
@ -448,7 +449,7 @@ message GetDataEncryptionKeyRequestProto { // no parameters
}
message GetDataEncryptionKeyResponseProto {
required DataEncryptionKeyProto dataEncryptionKey = 1;
optional DataEncryptionKeyProto dataEncryptionKey = 1;
}
message CreateSnapshotRequestProto {

View File

@ -24,6 +24,7 @@ option java_outer_classname = "DataTransferProtos";
option java_generate_equals_and_hash = true;
package hadoop.hdfs;
import "Security.proto";
import "hdfs.proto";
message DataTransferEncryptorMessageProto {
@ -39,7 +40,7 @@ message DataTransferEncryptorMessageProto {
message BaseHeaderProto {
required ExtendedBlockProto block = 1;
optional BlockTokenIdentifierProto token = 2;
optional hadoop.common.TokenProto token = 2;
}
message ClientOperationHeaderProto {

View File

@ -19,11 +19,14 @@
// This file contains protocol buffers that are used throughout HDFS -- i.e.
// by the client, server, and data transfer protocols.
option java_package = "org.apache.hadoop.hdfs.protocol.proto";
option java_outer_classname = "HdfsProtos";
option java_generate_equals_and_hash = true;
package hadoop.hdfs;
import "Security.proto";
/**
* Extended block idenfies a block
*/
@ -35,16 +38,6 @@ message ExtendedBlockProto {
// here for historical reasons
}
/**
* Block Token
*/
message BlockTokenIdentifierProto {
required bytes identifier = 1;
required bytes password = 2;
required string kind = 3;
required string service = 4;
}
/**
* Identifies a Datanode
*/
@ -126,7 +119,7 @@ message LocatedBlockProto {
// If block has few corrupt replicas, they are filtered and
// their locations are not part of this object
required BlockTokenIdentifierProto blockToken = 5;
required hadoop.common.TokenProto blockToken = 5;
}
message DataEncryptionKeyProto {

Some files were not shown because too many files have changed in this diff Show More