mirror of https://github.com/apache/druid.git
Fix resource leaks (ComplexColumn and GenericColumn) (#3629)
* Remove unused ComplexColumnImpl class * Remove throws IOException from close() in GenericColumn, ComplexColumn, IndexedFloats and IndexedLongs * Use concise try-with-resources syntax in several places * Fix resource leaks (ComplexColumn and GenericColumn) in SegmentAnalyzer, SearchQueryRunner, QueryableIndexIndexableAdapter and QueryableIndexStorageAdapter * Use Closer in Iterable, returned from QueryableIndexIndexableAdapter.getRows(), in order to try to close everything even if closing some parts thew exceptions
This commit is contained in:
parent
eb70a12e43
commit
4b0d6cf789
|
@ -306,35 +306,36 @@ public class SegmentAnalyzer
|
|||
final String typeName
|
||||
)
|
||||
{
|
||||
final ComplexColumn complexColumn = column != null ? column.getComplexColumn() : null;
|
||||
final boolean hasMultipleValues = capabilities != null && capabilities.hasMultipleValues();
|
||||
long size = 0;
|
||||
try (final ComplexColumn complexColumn = column != null ? column.getComplexColumn() : null) {
|
||||
final boolean hasMultipleValues = capabilities != null && capabilities.hasMultipleValues();
|
||||
long size = 0;
|
||||
|
||||
if (analyzingSize() && complexColumn != null) {
|
||||
final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName);
|
||||
if (serde == null) {
|
||||
return ColumnAnalysis.error(String.format("unknown_complex_%s", typeName));
|
||||
if (analyzingSize() && complexColumn != null) {
|
||||
final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName);
|
||||
if (serde == null) {
|
||||
return ColumnAnalysis.error(String.format("unknown_complex_%s", typeName));
|
||||
}
|
||||
|
||||
final Function<Object, Long> inputSizeFn = serde.inputSizeFn();
|
||||
if (inputSizeFn == null) {
|
||||
return new ColumnAnalysis(typeName, hasMultipleValues, 0, null, null, null, null);
|
||||
}
|
||||
|
||||
final int length = column.getLength();
|
||||
for (int i = 0; i < length; ++i) {
|
||||
size += inputSizeFn.apply(complexColumn.getRowValue(i));
|
||||
}
|
||||
}
|
||||
|
||||
final Function<Object, Long> inputSizeFn = serde.inputSizeFn();
|
||||
if (inputSizeFn == null) {
|
||||
return new ColumnAnalysis(typeName, hasMultipleValues, 0, null, null, null, null);
|
||||
}
|
||||
|
||||
final int length = column.getLength();
|
||||
for (int i = 0; i < length; ++i) {
|
||||
size += inputSizeFn.apply(complexColumn.getRowValue(i));
|
||||
}
|
||||
return new ColumnAnalysis(
|
||||
typeName,
|
||||
hasMultipleValues,
|
||||
size,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
return new ColumnAnalysis(
|
||||
typeName,
|
||||
hasMultipleValues,
|
||||
size,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -121,18 +121,22 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
|
|||
if (!interval.contains(segment.getDataInterval())) {
|
||||
MutableBitmap timeBitmap = bitmapFactory.makeEmptyMutableBitmap();
|
||||
final Column timeColumn = index.getColumn(Column.TIME_COLUMN_NAME);
|
||||
final GenericColumn timeValues = timeColumn.getGenericColumn();
|
||||
try (final GenericColumn timeValues = timeColumn.getGenericColumn()) {
|
||||
|
||||
int startIndex = Math.max(0, getStartIndexOfTime(timeValues, interval.getStartMillis(), true));
|
||||
int endIndex = Math.min(timeValues.length() - 1, getStartIndexOfTime(timeValues, interval.getEndMillis(), false));
|
||||
int startIndex = Math.max(0, getStartIndexOfTime(timeValues, interval.getStartMillis(), true));
|
||||
int endIndex = Math.min(
|
||||
timeValues.length() - 1,
|
||||
getStartIndexOfTime(timeValues, interval.getEndMillis(), false)
|
||||
);
|
||||
|
||||
for (int i = startIndex; i <= endIndex; i++) {
|
||||
timeBitmap.add(i);
|
||||
for (int i = startIndex; i <= endIndex; i++) {
|
||||
timeBitmap.add(i);
|
||||
}
|
||||
|
||||
final ImmutableBitmap finalTimeBitmap = bitmapFactory.makeImmutableBitmap(timeBitmap);
|
||||
timeFilteredBitmap =
|
||||
(baseFilter == null) ? finalTimeBitmap : finalTimeBitmap.intersection(baseFilter);
|
||||
}
|
||||
|
||||
final ImmutableBitmap finalTimeBitmap = bitmapFactory.makeImmutableBitmap(timeBitmap);
|
||||
timeFilteredBitmap =
|
||||
(baseFilter == null) ? finalTimeBitmap : finalTimeBitmap.intersection(baseFilter);
|
||||
} else {
|
||||
timeFilteredBitmap = baseFilter;
|
||||
}
|
||||
|
|
|
@ -97,14 +97,9 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
|||
@Override
|
||||
public int getNumRows()
|
||||
{
|
||||
GenericColumn column = null;
|
||||
try {
|
||||
column = index.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn();
|
||||
try (final GenericColumn column = index.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn()) {
|
||||
return column.length();
|
||||
}
|
||||
finally {
|
||||
CloseQuietly.close(column);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.base.Function;
|
|||
import com.google.common.collect.FluentIterable;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.io.Closer;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.guava.CloseQuietly;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
@ -178,8 +179,9 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
|||
return new Iterator<Rowboat>()
|
||||
{
|
||||
final GenericColumn timestamps = input.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn();
|
||||
final Object[] metrics;
|
||||
final Closeable[] metrics;
|
||||
final Closeable[] columns;
|
||||
final Closer closer = Closer.create();
|
||||
|
||||
final int numMetrics = getMetricNames().size();
|
||||
|
||||
|
@ -190,6 +192,8 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
|||
boolean done = false;
|
||||
|
||||
{
|
||||
closer.register(timestamps);
|
||||
|
||||
handlerSet.toArray(handlers);
|
||||
this.columns = FluentIterable
|
||||
.from(handlerSet)
|
||||
|
@ -204,9 +208,12 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
|||
}
|
||||
}
|
||||
).toArray(Closeable.class);
|
||||
for (Closeable column : columns) {
|
||||
closer.register(column);
|
||||
}
|
||||
|
||||
final Indexed<String> availableMetrics = getMetricNames();
|
||||
metrics = new Object[availableMetrics.size()];
|
||||
metrics = new Closeable[availableMetrics.size()];
|
||||
for (int i = 0; i < metrics.length; ++i) {
|
||||
final Column column = input.getColumn(availableMetrics.get(i));
|
||||
final ValueType type = column.getCapabilities().getType();
|
||||
|
@ -222,6 +229,9 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
|||
throw new ISE("Cannot handle type[%s]", type);
|
||||
}
|
||||
}
|
||||
for (Closeable metricColumn : metrics) {
|
||||
closer.register(metricColumn);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -229,15 +239,7 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
|||
{
|
||||
final boolean hasNext = currRow < numRows;
|
||||
if (!hasNext && !done) {
|
||||
CloseQuietly.close(timestamps);
|
||||
for (Object metric : metrics) {
|
||||
if (metric instanceof Closeable) {
|
||||
CloseQuietly.close((Closeable) metric);
|
||||
}
|
||||
}
|
||||
for (Closeable dimension : columns) {
|
||||
CloseQuietly.close(dimension);
|
||||
}
|
||||
CloseQuietly.close(closer);
|
||||
done = true;
|
||||
}
|
||||
return hasNext;
|
||||
|
@ -315,8 +317,11 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
|||
return "float";
|
||||
case LONG:
|
||||
return "long";
|
||||
case COMPLEX:
|
||||
return column.getComplexColumn().getTypeName();
|
||||
case COMPLEX: {
|
||||
try (ComplexColumn complexColumn = column.getComplexColumn() ) {
|
||||
return complexColumn.getTypeName();
|
||||
}
|
||||
}
|
||||
default:
|
||||
throw new ISE("Unknown type[%s]", type);
|
||||
}
|
||||
|
|
|
@ -28,13 +28,13 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.io.Closer;
|
||||
import com.metamx.collections.bitmap.ImmutableBitmap;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.math.expr.Expr;
|
||||
import io.druid.math.expr.Parser;
|
||||
import io.druid.java.util.common.guava.CloseQuietly;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.math.expr.Expr;
|
||||
import io.druid.math.expr.Parser;
|
||||
import io.druid.query.QueryInterruptedException;
|
||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
|
@ -135,27 +135,17 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public DateTime getMinTime()
|
||||
{
|
||||
GenericColumn column = null;
|
||||
try {
|
||||
column = index.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn();
|
||||
try (final GenericColumn column = index.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn()) {
|
||||
return new DateTime(column.getLongSingleValueRow(0));
|
||||
}
|
||||
finally {
|
||||
CloseQuietly.close(column);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DateTime getMaxTime()
|
||||
{
|
||||
GenericColumn column = null;
|
||||
try {
|
||||
column = index.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn();
|
||||
try (final GenericColumn column = index.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn()) {
|
||||
return new DateTime(column.getLongSingleValueRow(column.length() - 1));
|
||||
}
|
||||
finally {
|
||||
CloseQuietly.close(column);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -202,8 +192,9 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
public String getColumnTypeName(String columnName)
|
||||
{
|
||||
final Column column = index.getColumn(columnName);
|
||||
final ComplexColumn complexColumn = column.getComplexColumn();
|
||||
return complexColumn != null ? complexColumn.getTypeName() : column.getCapabilities().getType().toString();
|
||||
try (final ComplexColumn complexColumn = column.getComplexColumn()) {
|
||||
return complexColumn != null ? complexColumn.getTypeName() : column.getCapabilities().getType().toString();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -382,11 +373,13 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
|
||||
final Map<String, DictionaryEncodedColumn> dictionaryColumnCache = Maps.newHashMap();
|
||||
final Map<String, GenericColumn> genericColumnCache = Maps.newHashMap();
|
||||
final Map<String, ComplexColumn> complexColumnCache = Maps.newHashMap();
|
||||
final Map<String, Object> objectColumnCache = Maps.newHashMap();
|
||||
|
||||
final GenericColumn timestamps = index.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn();
|
||||
|
||||
final Closer closer = Closer.create();
|
||||
closer.register(timestamps);
|
||||
|
||||
Iterable<Long> iterable = gran.iterable(interval.getStartMillis(), interval.getEndMillis());
|
||||
if (descending) {
|
||||
iterable = Lists.reverse(ImmutableList.copyOf(iterable));
|
||||
|
@ -471,6 +464,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
DictionaryEncodedColumn<String> cachedColumn = dictionaryColumnCache.get(dimension);
|
||||
if (cachedColumn == null) {
|
||||
cachedColumn = columnDesc.getDictionaryEncoding();
|
||||
closer.register(cachedColumn);
|
||||
dictionaryColumnCache.put(dimension, cachedColumn);
|
||||
}
|
||||
|
||||
|
@ -592,6 +586,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
if (holder != null && (holder.getCapabilities().getType() == ValueType.FLOAT
|
||||
|| holder.getCapabilities().getType() == ValueType.LONG)) {
|
||||
cachedMetricVals = holder.getGenericColumn();
|
||||
closer.register(cachedMetricVals);
|
||||
genericColumnCache.put(columnName, cachedMetricVals);
|
||||
}
|
||||
}
|
||||
|
@ -628,6 +623,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
if (holder != null && (holder.getCapabilities().getType() == ValueType.LONG
|
||||
|| holder.getCapabilities().getType() == ValueType.FLOAT)) {
|
||||
cachedMetricVals = holder.getGenericColumn();
|
||||
closer.register(cachedMetricVals);
|
||||
genericColumnCache.put(columnName, cachedMetricVals);
|
||||
}
|
||||
}
|
||||
|
@ -676,6 +672,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
}
|
||||
|
||||
if (cachedColumnVals != null) {
|
||||
closer.register((Closeable) cachedColumnVals);
|
||||
objectColumnCache.put(column, cachedColumnVals);
|
||||
}
|
||||
}
|
||||
|
@ -822,6 +819,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
if (column == null) {
|
||||
continue;
|
||||
}
|
||||
closer.register(column);
|
||||
if (column.getType() == ValueType.FLOAT) {
|
||||
values.put(
|
||||
columnName, new Supplier<Number>()
|
||||
|
@ -1007,28 +1005,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
}
|
||||
}
|
||||
),
|
||||
new Closeable()
|
||||
{
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
CloseQuietly.close(timestamps);
|
||||
for (DictionaryEncodedColumn column : dictionaryColumnCache.values()) {
|
||||
CloseQuietly.close(column);
|
||||
}
|
||||
for (GenericColumn column : genericColumnCache.values()) {
|
||||
CloseQuietly.close(column);
|
||||
}
|
||||
for (ComplexColumn complexColumn : complexColumnCache.values()) {
|
||||
CloseQuietly.close(complexColumn);
|
||||
}
|
||||
for (Object column : objectColumnCache.values()) {
|
||||
if (column instanceof Closeable) {
|
||||
CloseQuietly.close((Closeable) column);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
closer
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,4 +28,7 @@ public interface ComplexColumn extends Closeable
|
|||
public Class<?> getClazz();
|
||||
public String getTypeName();
|
||||
public Object getRowValue(int rowNum);
|
||||
|
||||
@Override
|
||||
void close();
|
||||
}
|
||||
|
|
|
@ -1,57 +0,0 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets 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 io.druid.segment.column;
|
||||
|
||||
import io.druid.segment.data.Indexed;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ComplexColumnImpl extends AbstractColumn
|
||||
{
|
||||
private static final ColumnCapabilitiesImpl CAPABILITIES = new ColumnCapabilitiesImpl()
|
||||
.setType(ValueType.COMPLEX);
|
||||
|
||||
private final Indexed column;
|
||||
private final String typeName;
|
||||
|
||||
public ComplexColumnImpl(String typeName, Indexed column)
|
||||
{
|
||||
this.column = column;
|
||||
this.typeName = typeName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnCapabilities getCapabilities()
|
||||
{
|
||||
return CAPABILITIES;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getLength()
|
||||
{
|
||||
return column.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ComplexColumn getComplexColumn()
|
||||
{
|
||||
return new IndexedComplexColumn(typeName, column);
|
||||
}
|
||||
}
|
|
@ -39,4 +39,7 @@ public interface GenericColumn extends Closeable
|
|||
public IndexedFloats getFloatMultiValueRow(int rowNum);
|
||||
public long getLongSingleValueRow(int rowNum);
|
||||
public IndexedLongs getLongMultiValueRow(int rowNum);
|
||||
|
||||
@Override
|
||||
void close();
|
||||
}
|
||||
|
|
|
@ -56,7 +56,7 @@ public class IndexedComplexColumn implements ComplexColumn
|
|||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
public void close()
|
||||
{
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,8 +23,6 @@ import io.druid.segment.data.Indexed;
|
|||
import io.druid.segment.data.IndexedFloats;
|
||||
import io.druid.segment.data.IndexedLongs;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class IndexedFloatsGenericColumn implements GenericColumn
|
||||
|
@ -92,7 +90,7 @@ public class IndexedFloatsGenericColumn implements GenericColumn
|
|||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
public void close()
|
||||
{
|
||||
column.close();
|
||||
}
|
||||
|
|
|
@ -23,8 +23,6 @@ import io.druid.segment.data.Indexed;
|
|||
import io.druid.segment.data.IndexedFloats;
|
||||
import io.druid.segment.data.IndexedLongs;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class IndexedLongsGenericColumn implements GenericColumn
|
||||
|
@ -92,7 +90,7 @@ public class IndexedLongsGenericColumn implements GenericColumn
|
|||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
public void close()
|
||||
{
|
||||
column.close();
|
||||
}
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package io.druid.segment.column;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import io.druid.java.util.common.guava.CloseQuietly;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -62,14 +61,9 @@ class SimpleColumn implements Column
|
|||
@Override
|
||||
public int getLength()
|
||||
{
|
||||
GenericColumn column = null;
|
||||
try {
|
||||
column = genericColumn.get();
|
||||
try (final GenericColumn column = genericColumn.get()) {
|
||||
return column.length();
|
||||
}
|
||||
finally {
|
||||
CloseQuietly.close(column);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -20,12 +20,10 @@
|
|||
package io.druid.segment.data;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.google.common.primitives.Floats;
|
||||
import io.druid.collections.ResourceHolder;
|
||||
import io.druid.java.util.common.guava.CloseQuietly;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.nio.FloatBuffer;
|
||||
|
@ -140,9 +138,11 @@ public class BlockLayoutIndexedFloatSupplier implements Supplier<IndexedFloats>
|
|||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
public void close()
|
||||
{
|
||||
Closeables.close(holder, false);
|
||||
if (holder != null) {
|
||||
holder.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,11 +20,9 @@
|
|||
package io.druid.segment.data;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.io.Closeables;
|
||||
import io.druid.collections.ResourceHolder;
|
||||
import io.druid.java.util.common.guava.CloseQuietly;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.nio.LongBuffer;
|
||||
|
@ -175,9 +173,11 @@ public class BlockLayoutIndexedLongSupplier implements Supplier<IndexedLongs>
|
|||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
public void close()
|
||||
{
|
||||
Closeables.close(holder, false);
|
||||
if (holder != null) {
|
||||
holder.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.segment.data;
|
|||
|
||||
import com.google.common.base.Supplier;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.nio.FloatBuffer;
|
||||
|
@ -82,7 +81,7 @@ public class EntireLayoutIndexedFloatSupplier implements Supplier<IndexedFloats>
|
|||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
public void close()
|
||||
{
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,8 +21,6 @@ package io.druid.segment.data;
|
|||
|
||||
import com.google.common.base.Supplier;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class EntireLayoutIndexedLongSupplier implements Supplier<IndexedLongs>
|
||||
{
|
||||
|
||||
|
@ -80,7 +78,7 @@ public class EntireLayoutIndexedLongSupplier implements Supplier<IndexedLongs>
|
|||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
public void close()
|
||||
{
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,4 +29,7 @@ public interface IndexedFloats extends Closeable
|
|||
public int size();
|
||||
public float get(int index);
|
||||
public void fill(int index, float[] toFill);
|
||||
|
||||
@Override
|
||||
void close();
|
||||
}
|
||||
|
|
|
@ -29,4 +29,7 @@ public interface IndexedLongs extends Closeable
|
|||
public int size();
|
||||
public long get(int index);
|
||||
public void fill(int index, long[] toFill);
|
||||
|
||||
@Override
|
||||
void close();
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue