Skip to content

Commit

Permalink
HDFS-11644. Support for querying outputstream capabilities. Contribut…
Browse files Browse the repository at this point in the history
…ed by Manoj Govindassamy.
  • Loading branch information
umbrant committed May 9, 2017
1 parent 749e5c0 commit 54fd0e4
Show file tree
Hide file tree
Showing 7 changed files with 150 additions and 14 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
@InterfaceAudience.Public
@InterfaceStability.Stable
public class FSDataOutputStream extends DataOutputStream
implements Syncable, CanSetDropBehind {
implements Syncable, CanSetDropBehind, StreamCapabilities {
private final OutputStream wrappedStream;

private static class PositionCache extends FilterOutputStream {
Expand Down Expand Up @@ -111,6 +111,14 @@ public OutputStream getWrappedStream() {
return wrappedStream;
}

@Override
public boolean hasCapability(String capability) {
if (wrappedStream instanceof StreamCapabilities) {
return ((StreamCapabilities) wrappedStream).hasCapability(capability);
}
return false;
}

@Override // Syncable
public void hflush() throws IOException {
if (wrappedStream instanceof Syncable) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/**
* 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.fs;

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

/**
* Interface to query streams for supported capabilities.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface StreamCapabilities {
/**
* Capabilities that a stream can support and be queried for.
*/
enum StreamCapability {
/**
* Stream hflush capability to flush out the data in client's buffer.
* Streams with this capability implement {@link Syncable} and support
* {@link Syncable#hflush()}.
*/
HFLUSH("hflush"),

/**
* Stream hsync capability to flush out the data in client's buffer and
* the disk device. Streams with this capability implement {@link Syncable}
* and support {@link Syncable#hsync()}.
*/
HSYNC("hsync");

private final String capability;

StreamCapability(String value) {
this.capability = value;
}

public final String getValue() {
return capability;
}
}

/**
* Query the stream for a specific capability.
*
* @param capability string to query the stream support for.
* @return True if the stream supports capability.
*/
boolean hasCapability(String capability);
}

Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,9 @@
*/
package org.apache.hadoop.hdfs;

import static org.apache.hadoop.fs.StreamCapabilities.StreamCapability.HFLUSH;
import static org.apache.hadoop.fs.StreamCapabilities.StreamCapability.HSYNC;

import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InterruptedIOException;
Expand All @@ -34,6 +37,7 @@
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileEncryptionInfo;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.Syncable;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
Expand Down Expand Up @@ -90,7 +94,7 @@
****************************************************************/
@InterfaceAudience.Private
public class DFSOutputStream extends FSOutputSummer
implements Syncable, CanSetDropBehind {
implements Syncable, CanSetDropBehind, StreamCapabilities {
static final Logger LOG = LoggerFactory.getLogger(DFSOutputStream.class);
/**
* Number of times to retry creating a file when there are transient
Expand Down Expand Up @@ -546,6 +550,15 @@ void endBlock() throws IOException {
}
}

@Override
public boolean hasCapability(String capability) {
if (capability.equalsIgnoreCase(HSYNC.getValue()) ||
capability.equalsIgnoreCase((HFLUSH.getValue()))) {
return true;
}
return false;
}

/**
* Flushes out to all replicas of the block. The data is in the buffers
* of the DNs but not necessarily in the DN's OS buffers.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
Expand Down Expand Up @@ -77,8 +78,8 @@
* Each stripe contains a sequence of cells.
*/
@InterfaceAudience.Private
public class DFSStripedOutputStream extends DFSOutputStream {

public class DFSStripedOutputStream extends DFSOutputStream
implements StreamCapabilities {
private static final ByteBufferPool BUFFER_POOL = new ElasticByteBufferPool();

static class MultipleBlockingQueue<T> {
Expand Down Expand Up @@ -809,6 +810,12 @@ private int stripeDataSize() {
return numDataBlocks * cellSize;
}

@Override
public boolean hasCapability(String capability) {
// StreamCapabilities like hsync / hflush are not supported yet.
return false;
}

@Override
public void hflush() {
// not supported yet
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,11 @@
*/
package org.apache.hadoop.hdfs;

import java.io.ByteArrayInputStream;
import java.io.DataOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.lang.reflect.Field;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
Expand All @@ -32,8 +34,10 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FsTracer;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.StreamCapabilities.StreamCapability;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DataStreamer.LastExceptionInStreamer;
import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
Expand All @@ -48,13 +52,16 @@
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.PathUtils;
import org.apache.htrace.core.SpanId;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;

import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.anyBoolean;
import static org.mockito.Matchers.anyLong;
import org.mockito.Mockito;
Expand Down Expand Up @@ -346,6 +353,24 @@ public void testEndLeaseCall() throws Exception {
verify(spyClient, times(1)).endFileLease(anyLong());
}

@Test
public void testStreamFlush() throws Exception {
FileSystem fs = cluster.getFileSystem();
FSDataOutputStream os = fs.create(new Path("/normal-file"));
// Verify output stream supports hsync() and hflush().
assertTrue("DFSOutputStream should support hflush()!",
os.hasCapability(StreamCapability.HFLUSH.getValue()));
assertTrue("DFSOutputStream should support hsync()!",
os.hasCapability(StreamCapability.HSYNC.getValue()));
byte[] bytes = new byte[1024];
InputStream is = new ByteArrayInputStream(bytes);
IOUtils.copyBytes(is, os, bytes.length);
os.hflush();
IOUtils.copyBytes(is, os, bytes.length);
os.hsync();
os.close();
}

@AfterClass
public static void tearDown() {
if (cluster != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hdfs;

import static org.apache.hadoop.fs.contract.ContractTestUtils.fail;
import static org.junit.Assert.assertFalse;

import java.io.ByteArrayInputStream;
import java.io.IOException;
Expand All @@ -29,6 +29,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.StreamCapabilities.StreamCapability;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
import org.apache.hadoop.io.IOUtils;
Expand Down Expand Up @@ -195,14 +196,19 @@ public void testFileMoreThanABlockGroup3() throws Exception {
public void testStreamFlush() throws Exception {
final byte[] bytes = StripedFileTestUtil.generateBytes(blockSize *
dataBlocks * 3 + cellSize * dataBlocks + cellSize + 123);
try (FSDataOutputStream os = fs.create(new Path("/ec-file-1"))) {
InputStream is = new ByteArrayInputStream(bytes);
IOUtils.copyBytes(is, os, bytes.length);
os.hflush();
os.hsync();
} catch (Exception e) {
fail("hflush()/hsync() on striped file output stream failed!", e);
}
FSDataOutputStream os = fs.create(new Path("/ec-file-1"));
assertFalse("DFSStripedOutputStream should not have hflush() " +
"capability yet!", os.hasCapability(
StreamCapability.HFLUSH.getValue()));
assertFalse("DFSStripedOutputStream should not have hsync() " +
"capability yet!", os.hasCapability(
StreamCapability.HSYNC.getValue()));
InputStream is = new ByteArrayInputStream(bytes);
IOUtils.copyBytes(is, os, bytes.length);
os.hflush();
IOUtils.copyBytes(is, os, bytes.length);
os.hsync();
os.close();
}

private void testOneFile(String src, int writeBytes) throws Exception {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.io.IOException;
import java.io.OutputStream;

import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.Syncable;

/**
Expand All @@ -30,12 +31,21 @@
* wrapped stream passed in to the constructor. This is required
* for HBase when wrapping a PageBlobOutputStream used as a write-ahead log.
*/
public class SyncableDataOutputStream extends DataOutputStream implements Syncable {
public class SyncableDataOutputStream extends DataOutputStream
implements Syncable, StreamCapabilities {

public SyncableDataOutputStream(OutputStream out) {
super(out);
}

@Override
public boolean hasCapability(String capability) {
if (out instanceof StreamCapabilities) {
return ((StreamCapabilities) out).hasCapability(capability);
}
return false;
}

@Override
public void hflush() throws IOException {
if (out instanceof Syncable) {
Expand Down

0 comments on commit 54fd0e4

Please sign in to comment.