Skip to content

Commit

Permalink
[hotfix] [core] Fix lots of checkstyle errors in core.fs
Browse files Browse the repository at this point in the history
  • Loading branch information
StephanEwen committed Nov 24, 2017
1 parent 1854a3d commit f04b32b
Show file tree
Hide file tree
Showing 16 changed files with 163 additions and 172 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -32,13 +32,13 @@
@Internal
public abstract class AbstractMultiFSDataInputStream extends FSDataInputStream {

/** Inner stream for the currently accessed segment of the virtual global stream */
/** Inner stream for the currently accessed segment of the virtual global stream. */
protected FSDataInputStream delegate;

/** Position in the virtual global stream */
/** Position in the virtual global stream. */
protected long totalPos;

/** Total available bytes in the virtual global stream */
/** Total available bytes in the virtual global stream. */
protected long totalAvailable;

public AbstractMultiFSDataInputStream() {
Expand All @@ -48,7 +48,7 @@ public AbstractMultiFSDataInputStream() {
@Override
public void seek(long desired) throws IOException {

if(desired == totalPos) {
if (desired == totalPos) {
return;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ public interface BlockLocation extends Comparable<BlockLocation> {

/**
* Get the list of hosts (hostname) hosting this block.
*
*
* @return A list of hosts (hostname) hosting this block.
* @throws IOException
* thrown if the list of hosts could not be retrieved
Expand All @@ -39,14 +39,14 @@ public interface BlockLocation extends Comparable<BlockLocation> {

/**
* Get the start offset of the file associated with this block.
*
*
* @return The start offset of the file associated with this block.
*/
long getOffset();

/**
* Get the length of the block.
*
*
* @return the length of the block
*/
long getLength();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,10 +29,10 @@

/**
* This class allows to register instances of {@link Closeable}, which are all closed if this registry is closed.
* <p>
* Registering to an already closed registry will throw an exception and close the provided {@link Closeable}
* <p>
* All methods in this class are thread-safe.
*
* <p>Registering to an already closed registry will throw an exception and close the provided {@link Closeable}
*
* <p>All methods in this class are thread-safe.
*/
@Internal
public class CloseableRegistry extends AbstractCloseableRegistry<Closeable, Object> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,8 @@
/**
* This class is a {@link org.apache.flink.util.WrappingProxy} for {@link FSDataInputStream} that is used to
* implement a safety net against unclosed streams.
* <p>
* See {@link SafetyNetCloseableRegistry} for more details on how this is utilized.
*
* <p>See {@link SafetyNetCloseableRegistry} for more details on how this is utilized.
*/
@Internal
public class ClosingFSDataInputStream
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,8 @@
/**
* This class is a {@link org.apache.flink.util.WrappingProxy} for {@link FSDataOutputStream} that is used to
* implement a safety net against unclosed streams.
* <p>
* See {@link SafetyNetCloseableRegistry} for more details on how this is utilized.
*
* <p>See {@link SafetyNetCloseableRegistry} for more details on how this is utilized.
*/
@Internal
public class ClosingFSDataOutputStream
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@

/**
* Interface for a data input stream to a file on a {@link FileSystem}.
*
*
* <p>This extends the {@link java.io.InputStream} with methods for accessing
* the stream's {@link #getPos() current position} and
* {@link #seek(long) seeking} to a desired position.
Expand All @@ -36,7 +36,7 @@ public abstract class FSDataInputStream extends InputStream {
/**
* Seek to the given offset from the start of the file. The next read() will be from that location.
* Can't seek past the end of the file.
*
*
* @param desired
* the desired offset
* @throws IOException Thrown if an error occurred while seeking inside the input stream.
Expand All @@ -47,7 +47,7 @@ public abstract class FSDataInputStream extends InputStream {
* Gets the current position in the input stream.
*
* @return current position in the input stream
* @throws IOException Thrown if an I/O error occurred in the underlying stream
* @throws IOException Thrown if an I/O error occurred in the underlying stream
* implementation while accessing the stream's position.
*/
public abstract long getPos() throws IOException;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
import java.io.IOException;

/**
* Simple forwarding wrapper around {@link FSDataInputStream}
* Simple forwarding wrapper around {@link FSDataInputStream}.
*/
@Internal
public class FSDataInputStreamWrapper extends FSDataInputStream implements WrappingProxy<FSDataInputStream> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,22 +26,22 @@
/**
* An output stream to a file that is created via a {@link FileSystem}.
* This class extends the base {@link java.io.OutputStream} with some additional important methods.
*
*
* <h2>Data Persistence Guarantees</h2>
*
* These streams are used to persistently store data, both for results of streaming applications
*
* <p>These streams are used to persistently store data, both for results of streaming applications
* and for fault tolerance and recovery. It is therefore crucial that the persistence semantics
* of these streams are well defined.
*
*
* <p>Please refer to the class-level docs of {@link FileSystem} for the definition of data persistence
* via Flink's FileSystem abstraction and the {@code FSDataOutputStream}.
*
*
* <h2>Thread Safety</h2>
*
* Implementations of the {@code FSDataOutputStream} are generally not assumed to be thread safe.
*
* <p>Implementations of the {@code FSDataOutputStream} are generally not assumed to be thread safe.
* Instances of {@code FSDataOutputStream} should not be passed between threads, because there
* are no guarantees about the order of visibility of operations across threads.
*
*
* @see FileSystem
* @see FSDataInputStream
*/
Expand All @@ -52,13 +52,13 @@ public abstract class FSDataOutputStream extends OutputStream {
* Gets the position of the stream (non-negative), defined as the number of bytes
* from the beginning of the file to the current writing position. The position
* corresponds to the zero-based index of the next byte that will be written.
*
*
* <p>This method must report accurately report the current position of the stream.
* Various components of the high-availability and recovery logic rely on the accurate
*
*
* @return The current position in the stream, defined as the number of bytes
* from the beginning of the file to the current writing position.
*
*
* @throws IOException Thrown if an I/O error occurs while obtaining the position from
* the stream implementation.
*/
Expand All @@ -68,14 +68,14 @@ public abstract class FSDataOutputStream extends OutputStream {
* Flushes the stream, writing any data currently buffered in stream implementation
* to the proper output stream. After this method has been called, the stream implementation
* must not hold onto any buffered data any more.
*
*
* <p>A completed flush does not mean that the data is necessarily persistent. Data
* persistence can is only assumed after calls to {@link #close()} or {@link #sync()}.
*
*
* <p>Implementation note: This overrides the method defined in {@link OutputStream}
* as abstract to force implementations of the {@code FSDataOutputStream} to implement
* this method directly.
*
*
* @throws IOException Thrown if an I/O error occurs while flushing the stream.
*/
public abstract void flush() throws IOException;
Expand All @@ -84,9 +84,7 @@ public abstract class FSDataOutputStream extends OutputStream {
* Flushes the data all the way to the persistent non-volatile storage (for example disks).
* The method behaves similar to the <i>fsync</i> function, forcing all data to
* be persistent on the devices.
*
* <p>
*
*
* @throws IOException Thrown if an I/O error occurs
*/
public abstract void sync() throws IOException;
Expand All @@ -95,20 +93,20 @@ public abstract class FSDataOutputStream extends OutputStream {
* Closes the output stream. After this method returns, the implementation must guarantee
* that all data written to the stream is persistent/visible, as defined in the
* {@link FileSystem class-level docs}.
*
*
* <p>The above implies that the method must block until persistence can be guaranteed.
* For example for distributed replicated file systems, the method must block until the
* replication quorum has been reached. If the calling thread is interrupted in the
* process, it must fail with an {@code IOException} to indicate that persistence cannot
* be guaranteed.
*
*
* <p>If this method throws an exception, the data in the stream cannot be assumed to be
* persistent.
*
*
* <p>Implementation note: This overrides the method defined in {@link OutputStream}
* as abstract to force implementations of the {@code FSDataOutputStream} to implement
* this method directly.
*
*
* @throws IOException Thrown, if an error occurred while closing the stream or guaranteeing
* that the data is persistent.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
import java.io.IOException;

/**
* Simple forwarding wrapper around {@link FSDataInputStream}
* Simple forwarding wrapper around {@link FSDataInputStream}.
*/
@Internal
public class FSDataOutputStreamWrapper extends FSDataOutputStream implements WrappingProxy<FSDataOutputStream> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@

/**
* A file input split provides information on a particular part of a file, possibly
* hosted on a distributed file system and replicated among several hosts.
* hosted on a distributed file system and replicated among several hosts.
*/
@Public
public class FileInputSplit extends LocatableInputSplit {
Expand All @@ -34,16 +34,16 @@ public class FileInputSplit extends LocatableInputSplit {
private final Path file;

/** The position of the first byte in the file to process. */
private long start;
private final long start;

/** The number of bytes in the file to process. */
private long length;
private final long length;

// --------------------------------------------------------------------------------------------

/**
* Constructs a split with host information.
*
*
* @param num
* the number of this input split
* @param file
Expand All @@ -57,17 +57,17 @@ public class FileInputSplit extends LocatableInputSplit {
*/
public FileInputSplit(int num, Path file, long start, long length, String[] hosts) {
super(num, hosts);

this.file = file;
this.start = start;
this.length = length;
}

// --------------------------------------------------------------------------------------------

/**
* Returns the path of the file containing this split's data.
*
*
* @return the path of the file containing this split's data.
*/
public Path getPath() {
Expand All @@ -76,7 +76,7 @@ public Path getPath() {

/**
* Returns the position of the first byte in the file to process.
*
*
* @return the position of the first byte in the file to process
*/
public long getStart() {
Expand All @@ -85,28 +85,28 @@ public long getStart() {

/**
* Returns the number of bytes in the file to process.
*
*
* @return the number of bytes in the file to process
*/
public long getLength() {
return length;
}

// --------------------------------------------------------------------------------------------

@Override
public int hashCode() {
return getSplitNumber() ^ (file == null ? 0 : file.hashCode());
}

@Override
public boolean equals(Object obj) {
if (obj == this) {
return true;
}
else if (obj != null && obj instanceof FileInputSplit && super.equals(obj)) {
FileInputSplit other = (FileInputSplit) obj;

return this.start == other.start &&
this.length == other.length &&
(this.file == null ? other.file == null : (other.file != null && this.file.equals(other.file)));
Expand All @@ -115,7 +115,7 @@ else if (obj != null && obj instanceof FileInputSplit && super.equals(obj)) {
return false;
}
}

@Override
public String toString() {
return "[" + getSplitNumber() + "] " + file + ":" + start + "+" + length;
Expand Down
Loading

0 comments on commit f04b32b

Please sign in to comment.