mirror of https://github.com/apache/druid.git
Bump commons-io from 2.5 to 2.6 (#8006)
* Bump commons-io from 2.5 to 2.6 * Update licenses.yaml * Address comments
This commit is contained in:
parent
176da53996
commit
1a3aa1cfc0
|
@ -56,7 +56,7 @@ public interface Firehose extends Closeable
|
||||||
*
|
*
|
||||||
* @return true if and when there is another row available, false if the stream has dried up
|
* @return true if and when there is another row available, false if the stream has dried up
|
||||||
*/
|
*/
|
||||||
boolean hasMore();
|
boolean hasMore() throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The next row available. Should only be called if hasMore returns true.
|
* The next row available. Should only be called if hasMore returns true.
|
||||||
|
@ -65,7 +65,7 @@ public interface Firehose extends Closeable
|
||||||
* @return The next row
|
* @return The next row
|
||||||
*/
|
*/
|
||||||
@Nullable
|
@Nullable
|
||||||
InputRow nextRow();
|
InputRow nextRow() throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns an InputRowPlusRaw object containing the InputRow plus the raw, unparsed data corresponding to the next row
|
* Returns an InputRowPlusRaw object containing the InputRow plus the raw, unparsed data corresponding to the next row
|
||||||
|
@ -75,7 +75,7 @@ public interface Firehose extends Closeable
|
||||||
*
|
*
|
||||||
* @return an InputRowPlusRaw which may contain any of: an InputRow, the raw data, or a ParseException
|
* @return an InputRowPlusRaw which may contain any of: an InputRow, the raw data, or a ParseException
|
||||||
*/
|
*/
|
||||||
default InputRowPlusRaw nextRowWithRaw()
|
default InputRowPlusRaw nextRowWithRaw() throws IOException
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
return InputRowPlusRaw.of(nextRow(), null);
|
return InputRowPlusRaw.of(nextRow(), null);
|
||||||
|
|
|
@ -62,7 +62,7 @@ public class FileIteratingFirehose implements Firehose
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean hasMore()
|
public boolean hasMore() throws IOException
|
||||||
{
|
{
|
||||||
while ((lineIterator == null || !lineIterator.hasNext()) && lineIterators.hasNext()) {
|
while ((lineIterator == null || !lineIterator.hasNext()) && lineIterators.hasNext()) {
|
||||||
lineIterator = getNextLineIterator();
|
lineIterator = getNextLineIterator();
|
||||||
|
@ -73,7 +73,7 @@ public class FileIteratingFirehose implements Firehose
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public InputRow nextRow()
|
public InputRow nextRow() throws IOException
|
||||||
{
|
{
|
||||||
if (!hasMore()) {
|
if (!hasMore()) {
|
||||||
throw new NoSuchElementException();
|
throw new NoSuchElementException();
|
||||||
|
@ -83,7 +83,7 @@ public class FileIteratingFirehose implements Firehose
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public InputRowPlusRaw nextRowWithRaw()
|
public InputRowPlusRaw nextRowWithRaw() throws IOException
|
||||||
{
|
{
|
||||||
if (!hasMore()) {
|
if (!hasMore()) {
|
||||||
throw new NoSuchElementException();
|
throw new NoSuchElementException();
|
||||||
|
@ -98,7 +98,7 @@ public class FileIteratingFirehose implements Firehose
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private LineIterator getNextLineIterator()
|
private LineIterator getNextLineIterator() throws IOException
|
||||||
{
|
{
|
||||||
if (lineIterator != null) {
|
if (lineIterator != null) {
|
||||||
lineIterator.close();
|
lineIterator.close();
|
||||||
|
@ -119,7 +119,7 @@ public class FileIteratingFirehose implements Firehose
|
||||||
public void close() throws IOException
|
public void close() throws IOException
|
||||||
{
|
{
|
||||||
try (Closeable ignore = closer;
|
try (Closeable ignore = closer;
|
||||||
Closeable ignore2 = lineIterator != null ? lineIterator::close : null) {
|
Closeable ignore2 = lineIterator) {
|
||||||
// close both via try-with-resources
|
// close both via try-with-resources
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -336,7 +336,7 @@ name: Apache Commons IO
|
||||||
license_category: binary
|
license_category: binary
|
||||||
module: java-core
|
module: java-core
|
||||||
license_name: Apache License version 2.0
|
license_name: Apache License version 2.0
|
||||||
version: 2.5
|
version: 2.6
|
||||||
libraries:
|
libraries:
|
||||||
- commons-io: commons-io
|
- commons-io: commons-io
|
||||||
|
|
||||||
|
|
2
pom.xml
2
pom.xml
|
@ -206,7 +206,7 @@
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>commons-io</groupId>
|
<groupId>commons-io</groupId>
|
||||||
<artifactId>commons-io</artifactId>
|
<artifactId>commons-io</artifactId>
|
||||||
<version>2.5</version>
|
<version>2.6</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>commons-logging</groupId>
|
<groupId>commons-logging</groupId>
|
||||||
|
|
|
@ -105,14 +105,14 @@ public class CombiningFirehoseFactory implements FirehoseFactory<InputRowParser>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean hasMore()
|
public boolean hasMore() throws IOException
|
||||||
{
|
{
|
||||||
return currentFirehose.hasMore();
|
return currentFirehose.hasMore();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public InputRow nextRow()
|
public InputRow nextRow() throws IOException
|
||||||
{
|
{
|
||||||
InputRow rv = currentFirehose.nextRow();
|
InputRow rv = currentFirehose.nextRow();
|
||||||
if (!currentFirehose.hasMore()) {
|
if (!currentFirehose.hasMore()) {
|
||||||
|
|
|
@ -67,17 +67,17 @@ public class FixedCountFirehoseFactory implements FirehoseFactory
|
||||||
return new Firehose()
|
return new Firehose()
|
||||||
{
|
{
|
||||||
private int i = 0;
|
private int i = 0;
|
||||||
private Firehose delegateFirehose = delegate.connect(parser, temporaryDirectory);
|
private final Firehose delegateFirehose = delegate.connect(parser, temporaryDirectory);
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean hasMore()
|
public boolean hasMore() throws IOException
|
||||||
{
|
{
|
||||||
return i < count && delegateFirehose.hasMore();
|
return i < count && delegateFirehose.hasMore();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public InputRow nextRow()
|
public InputRow nextRow() throws IOException
|
||||||
{
|
{
|
||||||
Preconditions.checkArgument(i++ < count, "Max events limit reached.");
|
Preconditions.checkArgument(i++ < count, "Max events limit reached.");
|
||||||
return delegateFirehose.nextRow();
|
return delegateFirehose.nextRow();
|
||||||
|
|
|
@ -50,7 +50,7 @@ public class PredicateFirehose implements Firehose
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean hasMore()
|
public boolean hasMore() throws IOException
|
||||||
{
|
{
|
||||||
if (savedInputRow != null) {
|
if (savedInputRow != null) {
|
||||||
return true;
|
return true;
|
||||||
|
|
|
@ -107,20 +107,20 @@ public class TimedShutoffFirehoseFactory implements FirehoseFactory<InputRowPars
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean hasMore()
|
public boolean hasMore() throws IOException
|
||||||
{
|
{
|
||||||
return firehose.hasMore();
|
return firehose.hasMore();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public InputRow nextRow()
|
public InputRow nextRow() throws IOException
|
||||||
{
|
{
|
||||||
return firehose.nextRow();
|
return firehose.nextRow();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public InputRowPlusRaw nextRowWithRaw()
|
public InputRowPlusRaw nextRowWithRaw() throws IOException
|
||||||
{
|
{
|
||||||
return firehose.nextRowWithRaw();
|
return firehose.nextRowWithRaw();
|
||||||
}
|
}
|
||||||
|
|
|
@ -30,6 +30,8 @@ import org.apache.druid.segment.incremental.IncrementalIndexAddResult;
|
||||||
import org.apache.druid.segment.incremental.IndexSizeExceededException;
|
import org.apache.druid.segment.incremental.IndexSizeExceededException;
|
||||||
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
public class Plumbers
|
public class Plumbers
|
||||||
{
|
{
|
||||||
private static final Logger log = new Logger(Plumbers.class);
|
private static final Logger log = new Logger(Plumbers.class);
|
||||||
|
@ -45,7 +47,7 @@ public class Plumbers
|
||||||
final Plumber plumber,
|
final Plumber plumber,
|
||||||
final boolean reportParseExceptions,
|
final boolean reportParseExceptions,
|
||||||
final FireDepartmentMetrics metrics
|
final FireDepartmentMetrics metrics
|
||||||
)
|
) throws IOException
|
||||||
{
|
{
|
||||||
final InputRow inputRow;
|
final InputRow inputRow;
|
||||||
try {
|
try {
|
||||||
|
|
Loading…
Reference in New Issue