diff --git a/parquet-avro/src/test/java/org/apache/parquet/avro/TestInputOutputFormat.java b/parquet-avro/src/test/java/org/apache/parquet/avro/TestInputOutputFormat.java index 36c090fb97..7ba6c9b8ec 100644 --- a/parquet-avro/src/test/java/org/apache/parquet/avro/TestInputOutputFormat.java +++ b/parquet-avro/src/test/java/org/apache/parquet/avro/TestInputOutputFormat.java @@ -36,14 +36,15 @@ import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; import org.junit.Test; -import org.apache.parquet.Log; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static java.lang.Thread.sleep; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; public class TestInputOutputFormat { - private static final Log LOG = Log.getLog(TestInputOutputFormat.class); + private static final Logger LOG = LoggerFactory.getLogger(TestInputOutputFormat.class); private static Schema avroSchema; static { @@ -132,10 +133,10 @@ public void testReadWrite() throws Exception { private void waitForJob(Job job) throws Exception { job.submit(); while (!job.isComplete()) { - LOG.debug("waiting for job " + job.getJobName()); + LOG.debug("waiting for job {}", job.getJobName()); sleep(100); } - LOG.info("status for job " + job.getJobName() + ": " + (job.isSuccessful() ? "SUCCESS" : "FAILURE")); + LOG.info("status for job {}: {}", job.getJobName(), (job.isSuccessful() ? "SUCCESS" : "FAILURE")); if (!job.isSuccessful()) { throw new RuntimeException("job failed " + job.getJobName()); } diff --git a/parquet-avro/src/test/java/org/apache/parquet/avro/TestReflectInputOutputFormat.java b/parquet-avro/src/test/java/org/apache/parquet/avro/TestReflectInputOutputFormat.java index 3e1d32eeab..729f24ac40 100644 --- a/parquet-avro/src/test/java/org/apache/parquet/avro/TestReflectInputOutputFormat.java +++ b/parquet-avro/src/test/java/org/apache/parquet/avro/TestReflectInputOutputFormat.java @@ -37,7 +37,6 @@ import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; -import org.apache.parquet.Log; import org.apache.parquet.column.ColumnReader; import org.apache.parquet.filter.ColumnPredicates; import org.apache.parquet.filter.ColumnRecordFilter; @@ -46,6 +45,8 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static java.lang.Thread.sleep; import static org.junit.Assert.assertArrayEquals; @@ -55,7 +56,7 @@ import static org.junit.Assert.fail; public class TestReflectInputOutputFormat { - private static final Log LOG = Log.getLog(TestReflectInputOutputFormat.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReflectInputOutputFormat.class); public static class Service { @@ -477,10 +478,10 @@ public void testReadWriteChangedCar() throws Exception { private void waitForJob(Job job) throws Exception { job.submit(); while (!job.isComplete()) { - LOG.debug("waiting for job " + job.getJobName()); + LOG.debug("waiting for job {}", job.getJobName()); sleep(100); } - LOG.info("status for job " + job.getJobName() + ": " + (job.isSuccessful() ? "SUCCESS" : "FAILURE")); + LOG.info("status for job {}: {}", job.getJobName(), (job.isSuccessful() ? "SUCCESS" : "FAILURE")); if (!job.isSuccessful()) { throw new RuntimeException("job failed " + job.getJobName()); } diff --git a/parquet-avro/src/test/java/org/apache/parquet/avro/TestSpecificInputOutputFormat.java b/parquet-avro/src/test/java/org/apache/parquet/avro/TestSpecificInputOutputFormat.java index 17a0af1ef2..a0b58f3cdc 100644 --- a/parquet-avro/src/test/java/org/apache/parquet/avro/TestSpecificInputOutputFormat.java +++ b/parquet-avro/src/test/java/org/apache/parquet/avro/TestSpecificInputOutputFormat.java @@ -39,15 +39,16 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; -import org.apache.parquet.Log; import org.apache.parquet.column.ColumnReader; import org.apache.parquet.filter.ColumnPredicates; import org.apache.parquet.filter.ColumnRecordFilter; import org.apache.parquet.filter.RecordFilter; import org.apache.parquet.filter.UnboundRecordFilter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TestSpecificInputOutputFormat { - private static final Log LOG = Log.getLog(TestSpecificInputOutputFormat.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSpecificInputOutputFormat.class); public static Car nextRecord(int i) { String vin = "1VXBR12EXCP000000"; @@ -268,10 +269,10 @@ public void testReadWriteChangedCar() throws Exception { private void waitForJob(Job job) throws Exception { job.submit(); while (!job.isComplete()) { - LOG.debug("waiting for job " + job.getJobName()); + LOG.debug("waiting for job {}", job.getJobName()); sleep(100); } - LOG.info("status for job " + job.getJobName() + ": " + (job.isSuccessful() ? "SUCCESS" : "FAILURE")); + LOG.info("status for job {}: {}", job.getJobName(), (job.isSuccessful() ? "SUCCESS" : "FAILURE")); if (!job.isSuccessful()) { throw new RuntimeException("job failed " + job.getJobName()); } diff --git a/parquet-column/src/main/java/org/apache/parquet/CorruptDeltaByteArrays.java b/parquet-column/src/main/java/org/apache/parquet/CorruptDeltaByteArrays.java index 258c9ee1fd..d19e489e4a 100644 --- a/parquet-column/src/main/java/org/apache/parquet/CorruptDeltaByteArrays.java +++ b/parquet-column/src/main/java/org/apache/parquet/CorruptDeltaByteArrays.java @@ -20,9 +20,11 @@ import org.apache.parquet.VersionParser.ParsedVersion; import org.apache.parquet.column.Encoding; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class CorruptDeltaByteArrays { - private static final Log LOG = Log.getLog(CorruptStatistics.class); + private static final Logger LOG = LoggerFactory.getLogger(CorruptStatistics.class); private static final SemanticVersion PARQUET_246_FIXED_VERSION = new SemanticVersion(1, 8, 0); @@ -43,7 +45,7 @@ public static boolean requiresSequentialReads(ParsedVersion version, Encoding en if (!version.hasSemanticVersion()) { LOG.warn("Requiring sequential reads because created_by did not " + - "contain a valid version (see PARQUET-246): " + version.version); + "contain a valid version (see PARQUET-246): {}", version.version); return true; } @@ -61,7 +63,7 @@ public static boolean requiresSequentialReads(SemanticVersion semver, Encoding e if (semver.compareTo(PARQUET_246_FIXED_VERSION) < 0) { LOG.info("Requiring sequential reads because this file was created " + - "prior to " + PARQUET_246_FIXED_VERSION + ". See PARQUET-246" ); + "prior to {}. See PARQUET-246", PARQUET_246_FIXED_VERSION ); return true; } @@ -75,8 +77,7 @@ public static boolean requiresSequentialReads(String createdBy, Encoding encodin } if (Strings.isNullOrEmpty(createdBy)) { - LOG.info("Requiring sequential reads because file version is empty. " + - "See PARQUET-246"); + LOG.info("Requiring sequential reads because file version is empty. See PARQUET-246"); return true; } diff --git a/parquet-column/src/main/java/org/apache/parquet/CorruptStatistics.java b/parquet-column/src/main/java/org/apache/parquet/CorruptStatistics.java index 3b9033850e..3e3aa3c0d5 100644 --- a/parquet-column/src/main/java/org/apache/parquet/CorruptStatistics.java +++ b/parquet-column/src/main/java/org/apache/parquet/CorruptStatistics.java @@ -24,6 +24,8 @@ import org.apache.parquet.VersionParser.ParsedVersion; import org.apache.parquet.VersionParser.VersionParseException; import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * There was a bug (PARQUET-251) that caused the statistics metadata @@ -35,7 +37,7 @@ public class CorruptStatistics { private static final AtomicBoolean alreadyLogged = new AtomicBoolean(false); - private static final Log LOG = Log.getLog(CorruptStatistics.class); + private static final Logger LOG = LoggerFactory.getLogger(CorruptStatistics.class); // the version in which the bug described by jira: PARQUET-251 was fixed // the bug involved writing invalid binary statistics, so stats written prior to this diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderImpl.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderImpl.java index 3fc327e3d9..931b4b157e 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderImpl.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderImpl.java @@ -19,7 +19,6 @@ package org.apache.parquet.column.impl; import static java.lang.String.format; -import static org.apache.parquet.Log.DEBUG; import static org.apache.parquet.Preconditions.checkNotNull; import static org.apache.parquet.column.ValuesType.DEFINITION_LEVEL; import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL; @@ -30,7 +29,6 @@ import java.nio.ByteBuffer; import org.apache.parquet.CorruptDeltaByteArrays; -import org.apache.parquet.Log; import org.apache.parquet.VersionParser.ParsedVersion; import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.bytes.BytesUtils; @@ -51,6 +49,8 @@ import org.apache.parquet.io.api.PrimitiveConverter; import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeNameConverter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * ColumnReader implementation @@ -59,7 +59,7 @@ * */ public class ColumnReaderImpl implements ColumnReader { - private static final Log LOG = Log.getLog(ColumnReaderImpl.class); + private static final Logger LOG = LoggerFactory.getLogger(ColumnReaderImpl.class); /** * binds the lower level page decoder to the record converter materializing the records @@ -523,7 +523,7 @@ private void readRepetitionAndDefinitionLevels() { private void checkRead() { if (isPageFullyConsumed()) { if (isFullyConsumed()) { - if (DEBUG) LOG.debug("end reached"); + LOG.debug("end reached"); repetitionLevel = 0; // the next repetition level return; } @@ -533,7 +533,7 @@ private void checkRead() { } private void readPage() { - if (DEBUG) LOG.debug("loading page"); + LOG.debug("loading page"); DataPage page = pageReader.readPage(); page.accept(new DataPage.Visitor() { @Override @@ -590,14 +590,14 @@ private void readPageV1(DataPageV1 page) { this.definitionLevelColumn = new ValuesReaderIntIterator(dlReader); try { ByteBuffer bytes = page.getBytes().toByteBuffer(); - if (DEBUG) LOG.debug("page size " + bytes.remaining() + " bytes and " + pageValueCount + " records"); - if (DEBUG) LOG.debug("reading repetition levels at 0"); + LOG.debug("page size {} bytes and {} records", bytes.remaining(), pageValueCount); + LOG.debug("reading repetition levels at 0"); rlReader.initFromPage(pageValueCount, bytes, 0); int next = rlReader.getNextOffset(); - if (DEBUG) LOG.debug("reading definition levels at " + next); + LOG.debug("reading definition levels at {}", next); dlReader.initFromPage(pageValueCount, bytes, next); next = dlReader.getNextOffset(); - if (DEBUG) LOG.debug("reading data at " + next); + LOG.debug("reading data at {}", next); initDataReader(page.getValueEncoding(), bytes, next, page.getValueCount()); } catch (IOException e) { throw new ParquetDecodingException("could not read page " + page + " in col " + path, e); @@ -608,7 +608,7 @@ private void readPageV2(DataPageV2 page) { this.repetitionLevelColumn = newRLEIterator(path.getMaxRepetitionLevel(), page.getRepetitionLevels()); this.definitionLevelColumn = newRLEIterator(path.getMaxDefinitionLevel(), page.getDefinitionLevels()); try { - if (DEBUG) LOG.debug("page data size " + page.getData().size() + " bytes and " + pageValueCount + " records"); + LOG.debug("page data size {} bytes and {} records", page.getData().size(), pageValueCount); initDataReader(page.getDataEncoding(), page.getData().toByteBuffer(), 0, page.getValueCount()); } catch (IOException e) { throw new ParquetDecodingException("could not read page " + page + " in col " + path, e); diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV1.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV1.java index dc6ebecb5a..c5b3884194 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV1.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV1.java @@ -22,7 +22,6 @@ import java.io.IOException; -import org.apache.parquet.Log; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.ColumnWriter; import org.apache.parquet.column.ParquetProperties; @@ -32,8 +31,8 @@ import org.apache.parquet.column.values.ValuesWriter; import org.apache.parquet.io.ParquetEncodingException; import org.apache.parquet.io.api.Binary; - -import static java.lang.Math.max; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Writes (repetition level, definition level, value) triplets and deals with writing pages to the underlying layer. @@ -42,8 +41,11 @@ * */ final class ColumnWriterV1 implements ColumnWriter { - private static final Log LOG = Log.getLog(ColumnWriterV1.class); - private static final boolean DEBUG = Log.DEBUG; + private static final Logger LOG = LoggerFactory.getLogger(ColumnWriterV1.class); + + // By default: Debugging disabled this way (using the "if (DEBUG)" IN the methods) to allow + // the java compiler (not the JIT) to remove the unused statements during build time. + private static final boolean DEBUG = false; private final ColumnDescriptor path; private final PageWriter pageWriter; @@ -74,7 +76,7 @@ public ColumnWriterV1(ColumnDescriptor path, PageWriter pageWriter, } private void log(Object value, int r, int d) { - LOG.debug(path + " " + value + " r:" + r + " d:" + d); + if (DEBUG) LOG.debug( "{} {} r:{} d:{}", path, value, r, d); } private void resetStatistics() { diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV2.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV2.java index 396d53a1a5..c6fd91b5eb 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV2.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV2.java @@ -18,15 +18,10 @@ */ package org.apache.parquet.column.impl; -import static java.lang.Math.max; -import static org.apache.parquet.bytes.BytesUtils.getWidthFromMaxInt; - import java.io.IOException; import org.apache.parquet.Ints; -import org.apache.parquet.Log; import org.apache.parquet.bytes.BytesInput; -import org.apache.parquet.bytes.CapacityByteArrayOutputStream; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.ColumnWriter; import org.apache.parquet.column.Encoding; @@ -38,6 +33,8 @@ import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridEncoder; import org.apache.parquet.io.ParquetEncodingException; import org.apache.parquet.io.api.Binary; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Writes (repetition level, definition level, value) triplets and deals with writing pages to the underlying layer. @@ -46,8 +43,11 @@ * */ final class ColumnWriterV2 implements ColumnWriter { - private static final Log LOG = Log.getLog(ColumnWriterV2.class); - private static final boolean DEBUG = Log.DEBUG; + private static final Logger LOG = LoggerFactory.getLogger(ColumnWriterV2.class); + + // By default: Debugging disabled this way (using the "if (DEBUG)" IN the methods) to allow + // the java compiler (not the JIT) to remove the unused statements during build time. + private static final boolean DEBUG = false; private final ColumnDescriptor path; private final PageWriter pageWriter; @@ -73,7 +73,7 @@ public ColumnWriterV2( } private void log(Object value, int r, int d) { - LOG.debug(path + " " + value + " r:" + r + " d:" + d); + LOG.debug("{} {} r:{} d:{}", path, value, r, d); } private void resetStatistics() { diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/BitPackingValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/BitPackingValuesReader.java index f540c392b5..a5608cbef2 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/BitPackingValuesReader.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/BitPackingValuesReader.java @@ -25,11 +25,12 @@ import java.nio.ByteBuffer; import org.apache.parquet.bytes.ByteBufferInputStream; -import org.apache.parquet.Log; import org.apache.parquet.bytes.BytesUtils; import org.apache.parquet.column.values.ValuesReader; import org.apache.parquet.column.values.bitpacking.BitPacking.BitPackingReader; import org.apache.parquet.io.ParquetDecodingException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * a column reader that packs the ints in the number of bits required based on the maximum size. @@ -38,7 +39,7 @@ * */ public class BitPackingValuesReader extends ValuesReader { - private static final Log LOG = Log.getLog(BitPackingValuesReader.class); + private static final Logger LOG = LoggerFactory.getLogger(BitPackingValuesReader.class); private ByteBufferInputStream in; private BitPackingReader bitPackingReader; @@ -73,7 +74,7 @@ public int readInteger() { public void initFromPage(int valueCount, ByteBuffer in, int offset) throws IOException { int effectiveBitLength = valueCount * bitsPerValue; int length = BytesUtils.paddedByteCountFromBits(effectiveBitLength); - if (Log.DEBUG) LOG.debug("reading " + length + " bytes for " + valueCount + " values of size " + bitsPerValue + " bits." ); + LOG.debug("reading {} bytes for {} values of size {} bits.", length, valueCount, bitsPerValue); this.in = new ByteBufferInputStream(in, offset, length); this.bitPackingReader = createBitPackingReader(bitsPerValue, this.in, valueCount); this.nextOffset = offset + length; diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPackingValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPackingValuesReader.java index f4c8c8efc5..7c19340c8d 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPackingValuesReader.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPackingValuesReader.java @@ -22,14 +22,15 @@ import java.util.Arrays; import java.nio.ByteBuffer; -import org.apache.parquet.Log; import org.apache.parquet.bytes.BytesUtils; import org.apache.parquet.column.values.ValuesReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ByteBitPackingValuesReader extends ValuesReader { private static final int VALUES_AT_A_TIME = 8; // because we're using unpack8Values() - private static final Log LOG = Log.getLog(ByteBitPackingValuesReader.class); + private static final Logger LOG = LoggerFactory.getLogger(ByteBitPackingValuesReader.class); private final int bitWidth; private final BytePacker packer; @@ -69,7 +70,7 @@ public void initFromPage(int valueCount, ByteBuffer page, int offset) throws IOException { int effectiveBitLength = valueCount * bitWidth; int length = BytesUtils.paddedByteCountFromBits(effectiveBitLength); // ceil - if (Log.DEBUG) LOG.debug("reading " + length + " bytes for " + valueCount + " values of size " + bitWidth + " bits." ); + LOG.debug("reading {} bytes for {} values of size {} bits.", length, valueCount, bitWidth); this.encoded = page; this.encodedPos = offset; this.decodedPosition = VALUES_AT_A_TIME - 1; diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesReader.java index 41f221d85a..d810ba8110 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesReader.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesReader.java @@ -18,15 +18,15 @@ */ package org.apache.parquet.column.values.deltalengthbytearray; -import static org.apache.parquet.Log.DEBUG; import java.io.IOException; import java.nio.ByteBuffer; -import org.apache.parquet.Log; import org.apache.parquet.column.values.ValuesReader; import org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesReader; import org.apache.parquet.io.api.Binary; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Reads binary data written by {@link DeltaLengthByteArrayValuesWriter} @@ -36,7 +36,7 @@ */ public class DeltaLengthByteArrayValuesReader extends ValuesReader { - private static final Log LOG = Log.getLog(DeltaLengthByteArrayValuesReader.class); + private static final Logger LOG = LoggerFactory.getLogger(DeltaLengthByteArrayValuesReader.class); private ValuesReader lengthReader; private ByteBuffer in; private int offset; @@ -48,7 +48,7 @@ public DeltaLengthByteArrayValuesReader() { @Override public void initFromPage(int valueCount, ByteBuffer in, int offset) throws IOException { - if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (in.limit() - offset)); + LOG.debug("init from page at offset {} for length {}", offset, (in.limit() - offset)); lengthReader.initFromPage(valueCount, in, offset); offset = lengthReader.getNextOffset(); this.in = in; diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesWriter.java index f7ad912fb5..118153ced9 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesWriter.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesWriter.java @@ -21,7 +21,6 @@ import java.io.IOException; import org.apache.parquet.bytes.ByteBufferAllocator; -import org.apache.parquet.Log; import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.bytes.CapacityByteArrayOutputStream; import org.apache.parquet.bytes.LittleEndianDataOutputStream; @@ -31,6 +30,8 @@ import org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesWriterForInteger; import org.apache.parquet.io.ParquetEncodingException; import org.apache.parquet.io.api.Binary; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Write lengths of byte-arrays using delta encoding, followed by concatenated byte-arrays @@ -44,7 +45,7 @@ */ public class DeltaLengthByteArrayValuesWriter extends ValuesWriter { - private static final Log LOG = Log.getLog(DeltaLengthByteArrayValuesWriter.class); + private static final Logger LOG = LoggerFactory.getLogger(DeltaLengthByteArrayValuesWriter.class); private ValuesWriter lengthWriter; private CapacityByteArrayOutputStream arrayOut; @@ -81,7 +82,7 @@ public BytesInput getBytes() { } catch (IOException e) { throw new ParquetEncodingException("could not write page", e); } - if (Log.DEBUG) LOG.debug("writing a buffer of size " + arrayOut.size()); + LOG.debug("writing a buffer of size {}", arrayOut.size()); return BytesInput.concat(lengthWriter.getBytes(), BytesInput.from(arrayOut)); } diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesReader.java index e421da99ff..19ff47c239 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesReader.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesReader.java @@ -18,19 +18,19 @@ */ package org.apache.parquet.column.values.dictionary; -import static org.apache.parquet.Log.DEBUG; import java.io.IOException; import java.nio.ByteBuffer; import org.apache.parquet.bytes.ByteBufferInputStream; -import org.apache.parquet.Log; import org.apache.parquet.bytes.BytesUtils; import org.apache.parquet.column.Dictionary; import org.apache.parquet.column.values.ValuesReader; import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder; import org.apache.parquet.io.ParquetDecodingException; import org.apache.parquet.io.api.Binary; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Reads values that have been dictionary encoded @@ -39,7 +39,7 @@ * */ public class DictionaryValuesReader extends ValuesReader { - private static final Log LOG = Log.getLog(DictionaryValuesReader.class); + private static final Logger LOG = LoggerFactory.getLogger(DictionaryValuesReader.class); private ByteBufferInputStream in; @@ -56,10 +56,9 @@ public void initFromPage(int valueCount, ByteBuffer page, int offset) throws IOException { this.in = new ByteBufferInputStream(page, offset, page.limit() - offset); if (page.limit() - offset > 0) { - if (DEBUG) - LOG.debug("init from page at offset " + offset + " for length " + (page.limit() - offset)); + LOG.debug("init from page at offset {} for length {}", offset, (page.limit() - offset)); int bitWidth = BytesUtils.readIntLittleEndianOnOneByte(in); - if (DEBUG) LOG.debug("bit width " + bitWidth); + LOG.debug("bit width {}", bitWidth); decoder = new RunLengthBitPackingHybridDecoder(bitWidth, in); } else { decoder = new RunLengthBitPackingHybridDecoder(1, in) { diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesWriter.java index 86edd79b13..5ef7712878 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesWriter.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesWriter.java @@ -18,7 +18,6 @@ */ package org.apache.parquet.column.values.dictionary; -import static org.apache.parquet.Log.DEBUG; import static org.apache.parquet.bytes.BytesInput.concat; import it.unimi.dsi.fastutil.doubles.Double2IntLinkedOpenHashMap; import it.unimi.dsi.fastutil.doubles.Double2IntMap; @@ -41,7 +40,6 @@ import java.util.List; import org.apache.parquet.bytes.ByteBufferAllocator; -import org.apache.parquet.Log; import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.bytes.BytesUtils; import org.apache.parquet.bytes.CapacityByteArrayOutputStream; @@ -56,6 +54,9 @@ import org.apache.parquet.io.ParquetEncodingException; import org.apache.parquet.io.api.Binary; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * Will attempt to encode values using a dictionary and fall back to plain encoding * if the dictionary gets too big @@ -64,7 +65,7 @@ * */ public abstract class DictionaryValuesWriter extends ValuesWriter implements RequiresFallback { - private static final Log LOG = Log.getLog(DictionaryValuesWriter.class); + private static final Logger LOG = LoggerFactory.getLogger(DictionaryValuesWriter.class); /* max entries allowed for the dictionary will fail over to plain encoding if reached */ private static final int MAX_DICTIONARY_ENTRIES = Integer.MAX_VALUE - 1; @@ -158,7 +159,7 @@ public long getAllocatedSize() { @Override public BytesInput getBytes() { int maxDicId = getDictionarySize() - 1; - if (DEBUG) LOG.debug("max dic id " + maxDicId); + LOG.debug("max dic id {}", maxDicId); int bitWidth = BytesUtils.getWidthFromMaxInt(maxDicId); int initialSlabSize = CapacityByteArrayOutputStream.initialSlabSizeHeuristic(MIN_INITIAL_SLAB_SIZE, maxDictionaryByteSize, 10); @@ -174,7 +175,7 @@ public BytesInput getBytes() { // encodes the bit width byte[] bytesHeader = new byte[] { (byte) bitWidth }; BytesInput rleEncodedBytes = encoder.toBytes(); - if (DEBUG) LOG.debug("rle encoded bytes " + rleEncodedBytes.size()); + LOG.debug("rle encoded bytes {}", rleEncodedBytes.size()); BytesInput bytes = concat(BytesInput.from(bytesHeader), rleEncodedBytes); // remember size of dictionary when we last wrote a page lastUsedDictionarySize = getDictionarySize(); diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BinaryPlainValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BinaryPlainValuesReader.java index 26f5e29796..82e555134c 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BinaryPlainValuesReader.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BinaryPlainValuesReader.java @@ -18,19 +18,19 @@ */ package org.apache.parquet.column.values.plain; -import static org.apache.parquet.Log.DEBUG; import java.io.IOException; import java.nio.ByteBuffer; -import org.apache.parquet.Log; import org.apache.parquet.bytes.BytesUtils; import org.apache.parquet.column.values.ValuesReader; import org.apache.parquet.io.ParquetDecodingException; import org.apache.parquet.io.api.Binary; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class BinaryPlainValuesReader extends ValuesReader { - private static final Log LOG = Log.getLog(BinaryPlainValuesReader.class); + private static final Logger LOG = LoggerFactory.getLogger(BinaryPlainValuesReader.class); private ByteBuffer in; private int offset; @@ -63,7 +63,7 @@ public void skip() { @Override public void initFromPage(int valueCount, ByteBuffer in, int offset) throws IOException { - if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (in.limit() - offset)); + LOG.debug("init from page at offset {} for length {}", offset, (in.limit() - offset)); this.in = in; this.offset = offset; } diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BooleanPlainValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BooleanPlainValuesReader.java index a279938c69..1f8fc2c35f 100755 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BooleanPlainValuesReader.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BooleanPlainValuesReader.java @@ -18,15 +18,15 @@ */ package org.apache.parquet.column.values.plain; -import static org.apache.parquet.Log.DEBUG; import static org.apache.parquet.column.values.bitpacking.Packer.LITTLE_ENDIAN; import java.io.IOException; import java.nio.ByteBuffer; -import org.apache.parquet.Log; import org.apache.parquet.column.values.ValuesReader; import org.apache.parquet.column.values.bitpacking.ByteBitPackingValuesReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * encodes boolean for the plain encoding: one bit at a time (0 = false) @@ -35,7 +35,7 @@ * */ public class BooleanPlainValuesReader extends ValuesReader { - private static final Log LOG = Log.getLog(BooleanPlainValuesReader.class); + private static final Logger LOG = LoggerFactory.getLogger(BooleanPlainValuesReader.class); private ByteBitPackingValuesReader in = new ByteBitPackingValuesReader(1, LITTLE_ENDIAN); @@ -64,7 +64,7 @@ public void skip() { */ @Override public void initFromPage(int valueCount, ByteBuffer in, int offset) throws IOException { - if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (in.limit() - offset)); + LOG.debug("init from page at offset {} for length {}", offset, (in.limit() - offset)); this.in.initFromPage(valueCount, in, offset); } diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesReader.java index 8496e7e4c9..7a14f811ec 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesReader.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesReader.java @@ -20,12 +20,11 @@ import java.io.IOException; import java.nio.ByteBuffer; -import org.apache.parquet.Log; import org.apache.parquet.column.values.ValuesReader; import org.apache.parquet.io.ParquetDecodingException; import org.apache.parquet.io.api.Binary; - -import static org.apache.parquet.Log.DEBUG; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * ValuesReader for FIXED_LEN_BYTE_ARRAY. @@ -33,7 +32,7 @@ * @author David Z. Chen */ public class FixedLenByteArrayPlainValuesReader extends ValuesReader { - private static final Log LOG = Log.getLog(FixedLenByteArrayPlainValuesReader.class); + private static final Logger LOG = LoggerFactory.getLogger(FixedLenByteArrayPlainValuesReader.class); private ByteBuffer in; private int offset; private int length; @@ -61,7 +60,7 @@ public void skip() { @Override public void initFromPage(int valueCount, ByteBuffer in, int offset) throws IOException { - if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (in.limit() - offset)); + LOG.debug("init from page at offset {} for length {}", offset, (in.limit() - offset)); this.in = in; this.offset = offset; } diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesWriter.java index 6ab2dea4b1..d7b2deb42f 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesWriter.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesWriter.java @@ -21,7 +21,6 @@ import java.io.IOException; import org.apache.parquet.bytes.ByteBufferAllocator; -import org.apache.parquet.Log; import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.bytes.CapacityByteArrayOutputStream; import org.apache.parquet.bytes.LittleEndianDataOutputStream; @@ -29,6 +28,8 @@ import org.apache.parquet.column.Encoding; import org.apache.parquet.io.ParquetEncodingException; import org.apache.parquet.io.api.Binary; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * ValuesWriter for FIXED_LEN_BYTE_ARRAY. @@ -36,7 +37,7 @@ * @author David Z. Chen */ public class FixedLenByteArrayPlainValuesWriter extends ValuesWriter { - private static final Log LOG = Log.getLog(PlainValuesWriter.class); + private static final Logger LOG = LoggerFactory.getLogger(PlainValuesWriter.class); private CapacityByteArrayOutputStream arrayOut; private LittleEndianDataOutputStream out; @@ -76,7 +77,7 @@ public BytesInput getBytes() { } catch (IOException e) { throw new ParquetEncodingException("could not write page", e); } - if (Log.DEBUG) LOG.debug("writing a buffer of size " + arrayOut.size()); + LOG.debug("writing a buffer of size {}", arrayOut.size()); return BytesInput.from(arrayOut); } diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesReader.java index c8fb303f12..e79cbb2e10 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesReader.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesReader.java @@ -18,16 +18,15 @@ */ package org.apache.parquet.column.values.plain; -import static org.apache.parquet.Log.DEBUG; - import java.io.IOException; import java.nio.ByteBuffer; import org.apache.parquet.bytes.ByteBufferInputStream; -import org.apache.parquet.Log; import org.apache.parquet.bytes.LittleEndianDataInputStream; import org.apache.parquet.column.values.ValuesReader; import org.apache.parquet.io.ParquetDecodingException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Plain encoding for float, double, int, long @@ -36,7 +35,7 @@ * */ abstract public class PlainValuesReader extends ValuesReader { - private static final Log LOG = Log.getLog(PlainValuesReader.class); + private static final Logger LOG = LoggerFactory.getLogger(PlainValuesReader.class); protected LittleEndianDataInputStream in; @@ -46,7 +45,7 @@ abstract public class PlainValuesReader extends ValuesReader { */ @Override public void initFromPage(int valueCount, ByteBuffer in, int offset) throws IOException { - if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (in.limit() - offset)); + LOG.debug("init from page at offset {} for length {}", offset , (in.limit() - offset)); this.in = new LittleEndianDataInputStream(toInputStream(in, offset)); } diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java index add5495a39..aa96cb63db 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java @@ -22,7 +22,6 @@ import java.nio.charset.Charset; import org.apache.parquet.bytes.ByteBufferAllocator; -import org.apache.parquet.Log; import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.bytes.CapacityByteArrayOutputStream; import org.apache.parquet.bytes.LittleEndianDataOutputStream; @@ -30,6 +29,8 @@ import org.apache.parquet.column.values.ValuesWriter; import org.apache.parquet.io.ParquetEncodingException; import org.apache.parquet.io.api.Binary; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Plain encoding except for booleans @@ -38,7 +39,7 @@ * */ public class PlainValuesWriter extends ValuesWriter { - private static final Log LOG = Log.getLog(PlainValuesWriter.class); + private static final Logger LOG = LoggerFactory.getLogger(PlainValuesWriter.class); public static final Charset CHARSET = Charset.forName("UTF-8"); @@ -117,7 +118,7 @@ public BytesInput getBytes() { } catch (IOException e) { throw new ParquetEncodingException("could not write page", e); } - if (Log.DEBUG) LOG.debug("writing a buffer of size " + arrayOut.size()); + if (LOG.isDebugEnabled()) LOG.debug("writing a buffer of size {}", arrayOut.size()); return BytesInput.from(arrayOut); } diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridDecoder.java b/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridDecoder.java index 1280e8d989..6daa349ec8 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridDecoder.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridDecoder.java @@ -18,7 +18,6 @@ */ package org.apache.parquet.column.values.rle; -import static org.apache.parquet.Log.DEBUG; import java.io.DataInputStream; import java.io.IOException; @@ -26,12 +25,13 @@ import java.nio.ByteBuffer; import org.apache.parquet.bytes.ByteBufferInputStream; -import org.apache.parquet.Log; import org.apache.parquet.Preconditions; import org.apache.parquet.bytes.BytesUtils; import org.apache.parquet.column.values.bitpacking.BytePacker; import org.apache.parquet.column.values.bitpacking.Packer; import org.apache.parquet.io.ParquetDecodingException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Decodes values written in the grammar described in {@link RunLengthBitPackingHybridEncoder} @@ -39,7 +39,7 @@ * @author Julien Le Dem */ public class RunLengthBitPackingHybridDecoder { - private static final Log LOG = Log.getLog(RunLengthBitPackingHybridDecoder.class); + private static final Logger LOG = LoggerFactory.getLogger(RunLengthBitPackingHybridDecoder.class); private static enum MODE { RLE, PACKED } @@ -53,7 +53,7 @@ private static enum MODE { RLE, PACKED } private int[] currentBuffer; public RunLengthBitPackingHybridDecoder(int bitWidth, InputStream in) { - if (DEBUG) LOG.debug("decoding bitWidth " + bitWidth); + LOG.debug("decoding bitWidth {}", bitWidth); Preconditions.checkArgument(bitWidth >= 0 && bitWidth <= 32, "bitWidth must be >= 0 and <= 32"); this.bitWidth = bitWidth; @@ -87,13 +87,13 @@ private void readNext() throws IOException { switch (mode) { case RLE: currentCount = header >>> 1; - if (DEBUG) LOG.debug("reading " + currentCount + " values RLE"); + LOG.debug("reading {} values RLE", currentCount); currentValue = BytesUtils.readIntLittleEndianPaddedOnBitWidth(in, bitWidth); break; case PACKED: int numGroups = header >>> 1; currentCount = numGroups * 8; - if (DEBUG) LOG.debug("reading " + currentCount + " values BIT PACKED"); + LOG.debug("reading {} values BIT PACKED", currentCount); currentBuffer = new int[currentCount]; // TODO: reuse a buffer byte[] bytes = new byte[numGroups * bitWidth]; // At the end of the file RLE data though, there might not be that many bytes left. diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridEncoder.java b/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridEncoder.java index 001d3f695e..5fba70a70c 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridEncoder.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridEncoder.java @@ -21,15 +21,15 @@ import java.io.IOException; import org.apache.parquet.bytes.ByteBufferAllocator; -import org.apache.parquet.Log; import org.apache.parquet.Preconditions; import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.bytes.BytesUtils; import org.apache.parquet.bytes.CapacityByteArrayOutputStream; import org.apache.parquet.column.values.bitpacking.BytePacker; import org.apache.parquet.column.values.bitpacking.Packer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import static org.apache.parquet.Log.DEBUG; /** * Encodes values using a combination of run length encoding and bit packing, @@ -60,7 +60,7 @@ * @author Alex Levenson */ public class RunLengthBitPackingHybridEncoder { - private static final Log LOG = Log.getLog(RunLengthBitPackingHybridEncoder.class); + private static final Logger LOG = LoggerFactory.getLogger(RunLengthBitPackingHybridEncoder.class); private final BytePacker packer; @@ -118,10 +118,8 @@ public class RunLengthBitPackingHybridEncoder { private boolean toBytesCalled; public RunLengthBitPackingHybridEncoder(int bitWidth, int initialCapacity, int pageSize, ByteBufferAllocator allocator) { - if (DEBUG) { - LOG.debug(String.format("Encoding: RunLengthBitPackingHybridEncoder with " - + "bithWidth: %d initialCapacity %d", bitWidth, initialCapacity)); - } + LOG.debug("Encoding: RunLengthBitPackingHybridEncoder with " + + "bithWidth: {} initialCapacity {}", bitWidth, initialCapacity); Preconditions.checkArgument(bitWidth >= 0 && bitWidth <= 32, "bitWidth must be >= 0 and <= 32"); diff --git a/parquet-column/src/main/java/org/apache/parquet/example/data/Group.java b/parquet-column/src/main/java/org/apache/parquet/example/data/Group.java index 3fb7d4d152..61f63170d6 100644 --- a/parquet-column/src/main/java/org/apache/parquet/example/data/Group.java +++ b/parquet-column/src/main/java/org/apache/parquet/example/data/Group.java @@ -18,14 +18,14 @@ */ package org.apache.parquet.example.data; -import org.apache.parquet.Log; import org.apache.parquet.example.data.simple.NanoTime; import org.apache.parquet.io.api.Binary; import org.apache.parquet.io.api.RecordConsumer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; abstract public class Group extends GroupValueSource { - private static final Log logger = Log.getLog(Group.class); - private static final boolean DEBUG = Log.DEBUG; + private static final Logger LOG = LoggerFactory.getLogger(Group.class); public void add(String field, int value) { add(getType().getFieldIndex(field), value); @@ -64,7 +64,9 @@ public void add(String field, Group value) { } public Group addGroup(String field) { - if (DEBUG) logger.debug("add group "+field+" to "+getType().getName()); + if (LOG.isDebugEnabled()) { + LOG.debug("add group {} to {}", field, getType().getName()); + } return addGroup(getType().getFieldIndex(field)); } diff --git a/parquet-column/src/main/java/org/apache/parquet/filter2/compat/FilterCompat.java b/parquet-column/src/main/java/org/apache/parquet/filter2/compat/FilterCompat.java index 2efcc390af..17bd2e19ce 100644 --- a/parquet-column/src/main/java/org/apache/parquet/filter2/compat/FilterCompat.java +++ b/parquet-column/src/main/java/org/apache/parquet/filter2/compat/FilterCompat.java @@ -18,10 +18,11 @@ */ package org.apache.parquet.filter2.compat; -import org.apache.parquet.Log; import org.apache.parquet.filter.UnboundRecordFilter; import org.apache.parquet.filter2.predicate.FilterPredicate; import org.apache.parquet.filter2.predicate.LogicalInverseRewriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static org.apache.parquet.Preconditions.checkArgument; import static org.apache.parquet.Preconditions.checkNotNull; @@ -40,7 +41,7 @@ * codebase. */ public class FilterCompat { - private static final Log LOG = Log.getLog(FilterCompat.class); + private static final Logger LOG = LoggerFactory.getLogger(FilterCompat.class); /** * Anyone wanting to use a {@link Filter} need only implement this interface, @@ -67,13 +68,13 @@ public static interface Filter { public static Filter get(FilterPredicate filterPredicate) { checkNotNull(filterPredicate, "filterPredicate"); - LOG.info("Filtering using predicate: " + filterPredicate); + LOG.info("Filtering using predicate: {}", filterPredicate); // rewrite the predicate to not include the not() operator FilterPredicate collapsedPredicate = LogicalInverseRewriter.rewrite(filterPredicate); if (!filterPredicate.equals(collapsedPredicate)) { - LOG.info("Predicate has been collapsed to: " + collapsedPredicate); + LOG.info("Predicate has been collapsed to: {}", collapsedPredicate); } return new FilterPredicateCompat(collapsedPredicate); diff --git a/parquet-column/src/main/java/org/apache/parquet/io/BaseRecordReader.java b/parquet-column/src/main/java/org/apache/parquet/io/BaseRecordReader.java index f2d88fc11f..8c7f39028f 100644 --- a/parquet-column/src/main/java/org/apache/parquet/io/BaseRecordReader.java +++ b/parquet-column/src/main/java/org/apache/parquet/io/BaseRecordReader.java @@ -18,17 +18,17 @@ */ package org.apache.parquet.io; -import static org.apache.parquet.Log.DEBUG; -import org.apache.parquet.Log; import org.apache.parquet.column.ColumnReadStore; import org.apache.parquet.io.RecordReaderImplementation.State; import org.apache.parquet.io.api.Binary; import org.apache.parquet.io.api.RecordConsumer; import org.apache.parquet.io.api.RecordMaterializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; // TODO(julien): this class appears to be unused -- can it be nuked? - todd public abstract class BaseRecordReader extends RecordReader { - private static final Log LOG = Log.getLog(BaseRecordReader.class); + private static final Logger LOG = LoggerFactory.getLogger(BaseRecordReader.class); public RecordConsumer recordConsumer; public RecordMaterializer recordMaterializer; @@ -48,11 +48,11 @@ public T read() { private int endIndex; protected void currentLevel(int currentLevel) { - if (DEBUG) LOG.debug("currentLevel: "+currentLevel); + LOG.debug("currentLevel: {}",currentLevel); } protected void log(String message) { - if (DEBUG) LOG.debug("bc: "+message); + LOG.debug("bc: {}", message); } final protected int getCaseId(int state, int currentLevel, int d, int nextR) { @@ -62,18 +62,18 @@ final protected int getCaseId(int state, int currentLevel, int d, int nextR) { final protected void startMessage() { // reset state endField = null; - if (DEBUG) LOG.debug("startMessage()"); + LOG.debug("startMessage()"); recordConsumer.startMessage(); } final protected void startGroup(String field, int index) { startField(field, index); - if (DEBUG) LOG.debug("startGroup()"); + LOG.debug("startGroup()"); recordConsumer.startGroup(); } private void startField(String field, int index) { - if (DEBUG) LOG.debug("startField("+field+","+index+")"); + LOG.debug("startField({},{})", field, index); if (endField != null && index == endIndex) { // skip the close/open tag endField = null; @@ -89,13 +89,13 @@ private void startField(String field, int index) { final protected void addPrimitiveINT64(String field, int index, long value) { startField(field, index); - if (DEBUG) LOG.debug("addLong("+value+")"); + LOG.debug("addLong({})", value); recordConsumer.addLong(value); endField(field, index); } private void endField(String field, int index) { - if (DEBUG) LOG.debug("endField("+field+","+index+")"); + LOG.debug("endField({},{})", field, index); if (endField != null) { recordConsumer.endField(endField, endIndex); } @@ -105,14 +105,14 @@ private void endField(String field, int index) { final protected void addPrimitiveBINARY(String field, int index, Binary value) { startField(field, index); - if (DEBUG) LOG.debug("addBinary("+value+")"); + LOG.debug("addBinary({})", value); recordConsumer.addBinary(value); endField(field, index); } final protected void addPrimitiveINT32(String field, int index, int value) { startField(field, index); - if (DEBUG) LOG.debug("addInteger("+value+")"); + LOG.debug("addInteger({})", value); recordConsumer.addInteger(value); endField(field, index); } @@ -123,7 +123,7 @@ final protected void endGroup(String field, int index) { recordConsumer.endField(endField, endIndex); endField = null; } - if (DEBUG) LOG.debug("endGroup()"); + LOG.debug("endGroup()"); recordConsumer.endGroup(); endField(field, index); } @@ -134,7 +134,7 @@ final protected void endMessage() { recordConsumer.endField(endField, endIndex); endField = null; } - if (DEBUG) LOG.debug("endMessage()"); + LOG.debug("endMessage()"); recordConsumer.endMessage(); } diff --git a/parquet-column/src/main/java/org/apache/parquet/io/ColumnIO.java b/parquet-column/src/main/java/org/apache/parquet/io/ColumnIO.java index 95a969e733..9c6e7298f3 100644 --- a/parquet-column/src/main/java/org/apache/parquet/io/ColumnIO.java +++ b/parquet-column/src/main/java/org/apache/parquet/io/ColumnIO.java @@ -22,7 +22,6 @@ import java.util.Arrays; import java.util.List; -import org.apache.parquet.Log; import org.apache.parquet.schema.Type; import org.apache.parquet.schema.Type.Repetition; @@ -34,8 +33,6 @@ */ abstract public class ColumnIO { - static final boolean DEBUG = Log.DEBUG; - private final GroupColumnIO parent; private final Type type; private final String name; diff --git a/parquet-column/src/main/java/org/apache/parquet/io/GroupColumnIO.java b/parquet-column/src/main/java/org/apache/parquet/io/GroupColumnIO.java index 1efe0d1414..14b8426e38 100644 --- a/parquet-column/src/main/java/org/apache/parquet/io/GroupColumnIO.java +++ b/parquet-column/src/main/java/org/apache/parquet/io/GroupColumnIO.java @@ -27,8 +27,9 @@ import java.util.List; import java.util.Map; -import org.apache.parquet.Log; import org.apache.parquet.schema.GroupType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Group level of the IO structure @@ -38,7 +39,7 @@ * */ public class GroupColumnIO extends ColumnIO { - private static final Log LOG = Log.getLog(GroupColumnIO.class); + private static final Logger LOG = LoggerFactory.getLogger(GroupColumnIO.class); private final Map childrenByName = new HashMap(); private final List children = new ArrayList(); diff --git a/parquet-column/src/main/java/org/apache/parquet/io/MessageColumnIO.java b/parquet-column/src/main/java/org/apache/parquet/io/MessageColumnIO.java index f962105354..67efdb3a37 100644 --- a/parquet-column/src/main/java/org/apache/parquet/io/MessageColumnIO.java +++ b/parquet-column/src/main/java/org/apache/parquet/io/MessageColumnIO.java @@ -25,12 +25,10 @@ import java.util.List; import java.util.Map; -import org.apache.parquet.Log; import org.apache.parquet.column.ColumnWriteStore; import org.apache.parquet.column.ColumnWriter; import org.apache.parquet.column.impl.ColumnReadStoreImpl; import org.apache.parquet.column.page.PageReadStore; -import org.apache.parquet.column.values.dictionary.IntList; import org.apache.parquet.filter.UnboundRecordFilter; import org.apache.parquet.filter2.compat.FilterCompat; import org.apache.parquet.filter2.compat.FilterCompat.Filter; @@ -49,6 +47,9 @@ import it.unimi.dsi.fastutil.ints.IntArrayList; import it.unimi.dsi.fastutil.ints.IntIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import static org.apache.parquet.Preconditions.checkNotNull; /** @@ -58,9 +59,9 @@ * @author Julien Le Dem */ public class MessageColumnIO extends GroupColumnIO { - private static final Log logger = Log.getLog(MessageColumnIO.class); + private static final Logger LOG = LoggerFactory.getLogger(MessageColumnIO.class); - private static final boolean DEBUG = Log.DEBUG; + private static final boolean DEBUG = LOG.isDebugEnabled(); private List leaves; @@ -261,20 +262,24 @@ public MessageColumnIORecordConsumer(ColumnWriteStore columns) { r = new int[maxDepth]; } - public void printState() { - log(currentLevel + ", " + fieldsWritten[currentLevel] + ": " + Arrays.toString(currentColumnIO.getFieldPath()) + " r:" + r[currentLevel]); - if (r[currentLevel] > currentColumnIO.getRepetitionLevel()) { - // sanity check - throw new InvalidRecordException(r[currentLevel] + "(r) > " + currentColumnIO.getRepetitionLevel() + " ( schema r)"); + private void printState() { + if (DEBUG) { + log(currentLevel + ", " + fieldsWritten[currentLevel] + ": " + Arrays.toString(currentColumnIO.getFieldPath()) + " r:" + r[currentLevel]); + if (r[currentLevel] > currentColumnIO.getRepetitionLevel()) { + // sanity check + throw new InvalidRecordException(r[currentLevel] + "(r) > " + currentColumnIO.getRepetitionLevel() + " ( schema r)"); + } } } - private void log(Object m) { - String indent = ""; - for (int i = 0; i < currentLevel; ++i) { - indent += " "; + private void log(Object message, Object...parameters) { + if (DEBUG) { + String indent = ""; + for (int i = 0; i < currentLevel; ++i) { + indent += " "; + } + LOG.debug(indent + message, parameters); } - logger.debug(indent + m); } @Override @@ -298,7 +303,7 @@ public void endMessage() { @Override public void startField(String field, int index) { try { - if (DEBUG) log("startField(" + field + ", " + index + ")"); + if (DEBUG) log("startField({}, {})", field, index); currentColumnIO = ((GroupColumnIO) currentColumnIO).getChild(index); emptyField = true; if (DEBUG) printState(); @@ -309,7 +314,7 @@ public void startField(String field, int index) { @Override public void endField(String field, int index) { - if (DEBUG) log("endField(" + field + ", " + index + ")"); + if (DEBUG) log("endField({}, {})",field ,index); currentColumnIO = currentColumnIO.getParent(); if (emptyField) { throw new ParquetEncodingException("empty fields are illegal, the field should be ommited completely instead"); @@ -326,8 +331,7 @@ private void writeNullForMissingFieldsAtCurrentLevel() { try { ColumnIO undefinedField = ((GroupColumnIO) currentColumnIO).getChild(i); int d = currentColumnIO.getDefinitionLevel(); - if (DEBUG) - log(Arrays.toString(undefinedField.getFieldPath()) + ".writeNull(" + r[currentLevel] + "," + d + ")"); + if (DEBUG) log(Arrays.toString(undefinedField.getFieldPath()) + ".writeNull(" + r[currentLevel] + "," + d + ")"); writeNull(undefinedField, r[currentLevel], d); } catch (RuntimeException e) { throw new ParquetEncodingException("error while writing nulls for fields of indexes " + i + " . current index: " + fieldsWritten[currentLevel], e); @@ -372,7 +376,7 @@ private void writeNullToLeaves(GroupColumnIO group) { private void setRepetitionLevel() { r[currentLevel] = currentColumnIO.getRepetitionLevel(); - if (DEBUG) log("r: " + r[currentLevel]); + if (DEBUG) log("r: {}", r[currentLevel]); } @Override @@ -428,7 +432,7 @@ private ColumnWriter getColumnWriter() { @Override public void addInteger(int value) { - if (DEBUG) log("addInt(" + value + ")"); + if (DEBUG) log("addInt({})", value); emptyField = false; getColumnWriter().write(value, r[currentLevel], currentColumnIO.getDefinitionLevel()); @@ -438,7 +442,7 @@ public void addInteger(int value) { @Override public void addLong(long value) { - if (DEBUG) log("addLong(" + value + ")"); + if (DEBUG) log("addLong({})", value); emptyField = false; getColumnWriter().write(value, r[currentLevel], currentColumnIO.getDefinitionLevel()); @@ -448,7 +452,7 @@ public void addLong(long value) { @Override public void addBoolean(boolean value) { - if (DEBUG) log("addBoolean(" + value + ")"); + if (DEBUG) log("addBoolean({})", value); emptyField = false; getColumnWriter().write(value, r[currentLevel], currentColumnIO.getDefinitionLevel()); @@ -458,7 +462,7 @@ public void addBoolean(boolean value) { @Override public void addBinary(Binary value) { - if (DEBUG) log("addBinary(" + value.length() + " bytes)"); + if (DEBUG) log("addBinary({} bytes)", value.length()); emptyField = false; getColumnWriter().write(value, r[currentLevel], currentColumnIO.getDefinitionLevel()); @@ -468,7 +472,7 @@ public void addBinary(Binary value) { @Override public void addFloat(float value) { - if (DEBUG) log("addFloat(" + value + ")"); + if (DEBUG) log("addFloat({})", value); emptyField = false; getColumnWriter().write(value, r[currentLevel], currentColumnIO.getDefinitionLevel()); @@ -478,7 +482,7 @@ public void addFloat(float value) { @Override public void addDouble(double value) { - if (DEBUG) log("addDouble(" + value + ")"); + if (DEBUG) log("addDouble({})", value); emptyField = false; getColumnWriter().write(value, r[currentLevel], currentColumnIO.getDefinitionLevel()); diff --git a/parquet-column/src/main/java/org/apache/parquet/io/RecordConsumerLoggingWrapper.java b/parquet-column/src/main/java/org/apache/parquet/io/RecordConsumerLoggingWrapper.java index 7a8b1c1a2e..b90e21698f 100644 --- a/parquet-column/src/main/java/org/apache/parquet/io/RecordConsumerLoggingWrapper.java +++ b/parquet-column/src/main/java/org/apache/parquet/io/RecordConsumerLoggingWrapper.java @@ -18,10 +18,12 @@ */ package org.apache.parquet.io; -import java.util.Arrays; -import org.apache.parquet.Log; import org.apache.parquet.io.api.Binary; import org.apache.parquet.io.api.RecordConsumer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; /** * This class can be used to wrap an actual RecordConsumer and log all calls @@ -30,8 +32,7 @@ * */ public class RecordConsumerLoggingWrapper extends RecordConsumer { - private static final Log logger = Log.getLog(RecordConsumerLoggingWrapper.class); - private static final boolean DEBUG = Log.DEBUG; + private static final Logger LOG = LoggerFactory.getLogger(RecordConsumerLoggingWrapper.class); private final RecordConsumer delegate; @@ -50,12 +51,12 @@ public RecordConsumerLoggingWrapper(RecordConsumer delegate) { */ @Override public void startField(String field, int index) { - if (DEBUG) logOpen(field); + logOpen(field); delegate.startField(field, index); } private void logOpen(String field) { - log("<"+field+">"); + log("<{}>", field); } private String indent() { @@ -66,8 +67,10 @@ private String indent() { return result.toString(); } - private void log(Object value) { - logger.debug(indent() + value); + private void log(Object value, Object ... parameters) { + if (LOG.isDebugEnabled()) { + LOG.debug(indent() + value, parameters); + } } /** @@ -75,8 +78,8 @@ private void log(Object value) { */ @Override public void startGroup() { - if (DEBUG) ++indent; - if (DEBUG) log(""); + ++indent; + log(""); delegate.startGroup(); } @@ -85,7 +88,7 @@ public void startGroup() { */ @Override public void addInteger(int value) { - if (DEBUG) log(value); + log(value); delegate.addInteger(value); } @@ -94,7 +97,7 @@ public void addInteger(int value) { */ @Override public void addLong(long value) { - if (DEBUG) log(value); + log(value); delegate.addLong(value); } @@ -103,7 +106,7 @@ public void addLong(long value) { */ @Override public void addBoolean(boolean value) { - if (DEBUG) log(value); + log(value); delegate.addBoolean(value); } @@ -112,7 +115,7 @@ public void addBoolean(boolean value) { */ @Override public void addBinary(Binary value) { - if (DEBUG) log(Arrays.toString(value.getBytesUnsafe())); + if (LOG.isDebugEnabled()) log(Arrays.toString(value.getBytesUnsafe())); delegate.addBinary(value); } @@ -121,7 +124,7 @@ public void addBinary(Binary value) { */ @Override public void addFloat(float value) { - if (DEBUG) log(value); + log(value); delegate.addFloat(value); } @@ -130,7 +133,7 @@ public void addFloat(float value) { */ @Override public void addDouble(double value) { - if (DEBUG) log(value); + log(value); delegate.addDouble(value); } @@ -139,7 +142,7 @@ public void addDouble(double value) { */ @Override public void flush() { - if (DEBUG) log(""); + log(""); delegate.flush(); } @@ -148,8 +151,8 @@ public void flush() { */ @Override public void endGroup() { - if (DEBUG) log(""); - if (DEBUG) --indent; + log(""); + --indent; delegate.endGroup(); } @@ -158,12 +161,12 @@ public void endGroup() { */ @Override public void endField(String field, int index) { - if (DEBUG) logClose(field); + logClose(field); delegate.endField(field, index); } private void logClose(String field) { - log(""); + log("", field); } /** @@ -171,7 +174,7 @@ private void logClose(String field) { */ @Override public void startMessage() { - if (DEBUG) log(""); + log(""); delegate.startMessage(); } @@ -181,7 +184,7 @@ public void startMessage() { @Override public void endMessage() { delegate.endMessage(); - if (DEBUG) log(""); + log(""); } } diff --git a/parquet-column/src/main/java/org/apache/parquet/io/RecordReaderImplementation.java b/parquet-column/src/main/java/org/apache/parquet/io/RecordReaderImplementation.java index 7a87cbb003..af7d4a5828 100644 --- a/parquet-column/src/main/java/org/apache/parquet/io/RecordReaderImplementation.java +++ b/parquet-column/src/main/java/org/apache/parquet/io/RecordReaderImplementation.java @@ -26,7 +26,6 @@ import java.util.List; import java.util.Map; -import org.apache.parquet.Log; import org.apache.parquet.column.ColumnReader; import org.apache.parquet.column.impl.ColumnReadStoreImpl; import org.apache.parquet.io.api.Converter; @@ -36,6 +35,8 @@ import org.apache.parquet.io.api.RecordMaterializer; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -45,7 +46,7 @@ * @param the type of the materialized record */ class RecordReaderImplementation extends RecordReader { - private static final Log LOG = Log.getLog(RecordReaderImplementation.class); + private static final Logger LOG = LoggerFactory.getLogger(RecordReaderImplementation.class); public static class Case { @@ -376,7 +377,7 @@ private RecordConsumer validator(RecordConsumer recordConsumer, boolean validati } private RecordConsumer wrap(RecordConsumer recordConsumer) { - if (Log.DEBUG) { + if (LOG.isDebugEnabled()) { return new RecordConsumerLoggingWrapper(recordConsumer); } return recordConsumer; diff --git a/parquet-column/src/main/java/org/apache/parquet/io/ValidatingRecordConsumer.java b/parquet-column/src/main/java/org/apache/parquet/io/ValidatingRecordConsumer.java index 46f0aaeadb..c27381a3bd 100644 --- a/parquet-column/src/main/java/org/apache/parquet/io/ValidatingRecordConsumer.java +++ b/parquet-column/src/main/java/org/apache/parquet/io/ValidatingRecordConsumer.java @@ -22,13 +22,14 @@ import java.util.Arrays; import java.util.Deque; -import org.apache.parquet.Log; import org.apache.parquet.io.api.Binary; import org.apache.parquet.io.api.RecordConsumer; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.Type; import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; import org.apache.parquet.schema.Type.Repetition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.*; @@ -40,8 +41,7 @@ * */ public class ValidatingRecordConsumer extends RecordConsumer { - private static final Log LOG = Log.getLog(ValidatingRecordConsumer.class); - private static final boolean DEBUG = Log.DEBUG; + private static final Logger LOG = LoggerFactory.getLogger(ValidatingRecordConsumer.class); private final RecordConsumer delegate; @@ -139,7 +139,7 @@ private void validate(PrimitiveTypeName p) { Type currentType = types.peek().asGroupType().getType(fields.peek()); int c = fieldValueCount.pop() + 1; fieldValueCount.push(c); - if (DEBUG) LOG.debug("validate " + p + " for " + currentType.getName()); + LOG.debug("validate {} for {}",p ,currentType.getName()); switch (currentType.getRepetition()) { case OPTIONAL: case REQUIRED: @@ -161,7 +161,7 @@ private void validate(PrimitiveTypeName... ptypes) { Type currentType = types.peek().asGroupType().getType(fields.peek()); int c = fieldValueCount.pop() + 1; fieldValueCount.push(c); - if (DEBUG) LOG.debug("validate " + Arrays.toString(ptypes) + " for " + currentType.getName()); + if (LOG.isDebugEnabled()) LOG.debug("validate " + Arrays.toString(ptypes) + " for " + currentType.getName()); switch (currentType.getRepetition()) { case OPTIONAL: case REQUIRED: diff --git a/parquet-column/src/main/java/org/apache/parquet/schema/MessageTypeParser.java b/parquet-column/src/main/java/org/apache/parquet/schema/MessageTypeParser.java index b7274c2c5c..f0c178af68 100644 --- a/parquet-column/src/main/java/org/apache/parquet/schema/MessageTypeParser.java +++ b/parquet-column/src/main/java/org/apache/parquet/schema/MessageTypeParser.java @@ -22,11 +22,12 @@ import java.util.Locale; import java.util.StringTokenizer; -import org.apache.parquet.Log; import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; import org.apache.parquet.schema.Type.Repetition; import org.apache.parquet.schema.Types.GroupBuilder; import org.apache.parquet.schema.Types.PrimitiveBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Parses a schema from a textual format similar to that described in the Dremel paper. @@ -34,7 +35,7 @@ * @author Julien Le Dem */ public class MessageTypeParser { - private static final Log LOG = Log.getLog(MessageTypeParser.class); + private static final Logger LOG = LoggerFactory.getLogger(MessageTypeParser.class); private static class Tokenizer { diff --git a/parquet-column/src/test/java/org/apache/parquet/column/mem/TestMemColumn.java b/parquet-column/src/test/java/org/apache/parquet/column/mem/TestMemColumn.java index 42c1776cc7..c855339c59 100644 --- a/parquet-column/src/test/java/org/apache/parquet/column/mem/TestMemColumn.java +++ b/parquet-column/src/test/java/org/apache/parquet/column/mem/TestMemColumn.java @@ -23,7 +23,6 @@ import org.apache.parquet.column.ParquetProperties; import org.junit.Test; -import org.apache.parquet.Log; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.ColumnReader; import org.apache.parquet.column.ColumnWriter; @@ -34,9 +33,11 @@ import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.MessageTypeParser; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TestMemColumn { - private static final Log LOG = Log.getLog(TestMemColumn.class); + private static final Logger LOG = LoggerFactory.getLogger(TestMemColumn.class); @Test public void testMemColumn() throws Exception { @@ -134,7 +135,7 @@ public void testMemColumnSeveralPagesRepeated() throws Exception { for (int i = 0; i < 837; i++) { int r = rs[i % rs.length]; int d = ds[i % ds.length]; - LOG.debug("write i: " + i); + LOG.debug("write i: {}", i); if (d == 2) { columnWriter.write((long)i, r, d); } else { @@ -148,7 +149,7 @@ public void testMemColumnSeveralPagesRepeated() throws Exception { for (int j = 0; j < columnReader.getTotalValueCount(); j++) { int r = rs[i % rs.length]; int d = ds[i % ds.length]; - LOG.debug("read i: " + i); + LOG.debug("read i: {}", i); assertEquals("r row " + i, r, columnReader.getCurrentRepetitionLevel()); assertEquals("d row " + i, d, columnReader.getCurrentDefinitionLevel()); if (d == 2) { diff --git a/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageReader.java b/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageReader.java index a6e8910de5..5373c9a45f 100644 --- a/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageReader.java +++ b/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageReader.java @@ -18,20 +18,20 @@ */ package org.apache.parquet.column.page.mem; -import static org.apache.parquet.Log.DEBUG; import static org.apache.parquet.Preconditions.checkNotNull; import java.util.Iterator; -import org.apache.parquet.Log; import org.apache.parquet.column.page.DictionaryPage; import org.apache.parquet.column.page.DataPage; import org.apache.parquet.column.page.PageReader; import org.apache.parquet.io.ParquetDecodingException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class MemPageReader implements PageReader { - private static final Log LOG = Log.getLog(MemPageReader.class); + private static final Logger LOG = LoggerFactory.getLogger(MemPageReader.class); private final long totalValueCount; private final Iterator pages; @@ -54,7 +54,7 @@ public long getTotalValueCount() { public DataPage readPage() { if (pages.hasNext()) { DataPage next = pages.next(); - if (DEBUG) LOG.debug("read page " + next); + LOG.debug("read page {}", next); return next; } else { throw new ParquetDecodingException("after last page"); diff --git a/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageStore.java b/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageStore.java index 219e5cdd13..cdde89490d 100644 --- a/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageStore.java +++ b/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageStore.java @@ -18,12 +18,6 @@ */ package org.apache.parquet.column.page.mem; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.parquet.Log; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.UnknownColumnException; import org.apache.parquet.column.page.DataPage; @@ -31,10 +25,17 @@ import org.apache.parquet.column.page.PageReader; import org.apache.parquet.column.page.PageWriteStore; import org.apache.parquet.column.page.PageWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; public class MemPageStore implements PageReadStore, PageWriteStore { - private static final Log LOG = Log.getLog(MemPageStore.class); + private static final Logger LOG = LoggerFactory.getLogger(MemPageStore.class); private Map pageWriters = new HashMap(); @@ -62,7 +63,7 @@ public PageReader getPageReader(ColumnDescriptor descriptor) { throw new UnknownColumnException(descriptor); } List pages = new ArrayList(pageWriter.getPages()); - if (Log.DEBUG) LOG.debug("initialize page reader with "+ pageWriter.getTotalValueCount() + " values and " + pages.size() + " pages"); + LOG.debug("initialize page reader with {} values and {} pages", pageWriter.getTotalValueCount(), pages.size()); return new MemPageReader(pageWriter.getTotalValueCount(), pages.iterator(), pageWriter.getDictionaryPage()); } diff --git a/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageWriter.java b/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageWriter.java index ddab636319..be3a0f9cb4 100644 --- a/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageWriter.java +++ b/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageWriter.java @@ -18,26 +18,26 @@ */ package org.apache.parquet.column.page.mem; -import static org.apache.parquet.Log.DEBUG; -import static org.apache.parquet.bytes.BytesInput.copy; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -import org.apache.parquet.Log; import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.column.Encoding; +import org.apache.parquet.column.page.DataPage; import org.apache.parquet.column.page.DataPageV1; import org.apache.parquet.column.page.DataPageV2; import org.apache.parquet.column.page.DictionaryPage; -import org.apache.parquet.column.page.DataPage; import org.apache.parquet.column.page.PageWriter; import org.apache.parquet.column.statistics.Statistics; import org.apache.parquet.io.ParquetEncodingException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.parquet.bytes.BytesInput.copy; public class MemPageWriter implements PageWriter { - private static final Log LOG = Log.getLog(MemPageWriter.class); + private static final Logger LOG = LoggerFactory.getLogger(MemPageWriter.class); private final List pages = new ArrayList(); private DictionaryPage dictionaryPage; @@ -53,7 +53,7 @@ public void writePage(BytesInput bytesInput, int valueCount, Statistics statisti memSize += bytesInput.size(); pages.add(new DataPageV1(BytesInput.copy(bytesInput), valueCount, (int)bytesInput.size(), statistics, rlEncoding, dlEncoding, valuesEncoding)); totalValueCount += valueCount; - if (DEBUG) LOG.debug("page written for " + bytesInput.size() + " bytes and " + valueCount + " records"); + LOG.debug("page written for {} bytes and {} records", bytesInput.size(), valueCount); } @Override @@ -67,8 +67,7 @@ public void writePageV2(int rowCount, int nullCount, int valueCount, memSize += size; pages.add(DataPageV2.uncompressed(rowCount, nullCount, valueCount, copy(repetitionLevels), copy(definitionLevels), dataEncoding, copy(data), statistics)); totalValueCount += valueCount; - if (DEBUG) LOG.debug("page written for " + size + " bytes and " + valueCount + " records"); - + LOG.debug("page written for {} bytes and {} records", size, valueCount); } @Override @@ -101,7 +100,7 @@ public void writeDictionaryPage(DictionaryPage dictionaryPage) throws IOExceptio } this.memSize += dictionaryPage.getBytes().size(); this.dictionaryPage = dictionaryPage.copy(); - if (DEBUG) LOG.debug("dictionary page written for " + dictionaryPage.getBytes().size() + " bytes and " + dictionaryPage.getDictionarySize() + " records"); + LOG.debug("dictionary page written for {} bytes and {} records", dictionaryPage.getBytes().size(), dictionaryPage.getDictionarySize()); } @Override diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPackingColumn.java b/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPackingColumn.java index aef259c509..d83628a941 100644 --- a/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPackingColumn.java +++ b/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPackingColumn.java @@ -28,12 +28,13 @@ import org.junit.Test; import org.apache.parquet.bytes.DirectByteBufferAllocator; -import org.apache.parquet.Log; import org.apache.parquet.column.values.ValuesReader; import org.apache.parquet.column.values.ValuesWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TestBitPackingColumn { - private static final Log LOG = Log.getLog(TestBitPackingColumn.class); + private static final Logger LOG = LoggerFactory.getLogger(TestBitPackingColumn.class); @Test public void testZero() throws IOException { @@ -163,7 +164,7 @@ public void testSeven() throws IOException { private void validateEncodeDecode(int bitLength, int[] vals, String expected) throws IOException { for (PACKING_TYPE type : PACKING_TYPE.values()) { - LOG.debug(type); + LOG.debug("{}", type); final int bound = (int)Math.pow(2, bitLength) - 1; ValuesWriter w = type.getWriter(bound); for (int i : vals) { @@ -171,7 +172,7 @@ private void validateEncodeDecode(int bitLength, int[] vals, String expected) th } byte[] bytes = w.getBytes().toByteArray(); LOG.debug("vals ("+bitLength+"): " + TestBitPacking.toString(vals)); - LOG.debug("bytes: " + TestBitPacking.toString(bytes)); + LOG.debug("bytes: {}", TestBitPacking.toString(bytes)); assertEquals(type.toString(), expected, TestBitPacking.toString(bytes)); ValuesReader r = type.getReader(bound); r.initFromPage(vals.length, ByteBuffer.wrap(bytes), 0); @@ -179,7 +180,7 @@ private void validateEncodeDecode(int bitLength, int[] vals, String expected) th for (int i = 0; i < result.length; i++) { result[i] = r.readInteger(); } - LOG.debug("result: " + TestBitPacking.toString(result)); + LOG.debug("result: {}", TestBitPacking.toString(result)); assertArrayEquals(type + " result: " + TestBitPacking.toString(result), vals, result); } } diff --git a/parquet-column/src/test/java/org/apache/parquet/io/PerfTest.java b/parquet-column/src/test/java/org/apache/parquet/io/PerfTest.java index e4687b1c3c..bf783df836 100644 --- a/parquet-column/src/test/java/org/apache/parquet/io/PerfTest.java +++ b/parquet-column/src/test/java/org/apache/parquet/io/PerfTest.java @@ -24,9 +24,6 @@ import static org.apache.parquet.example.Paper.schema2; import static org.apache.parquet.example.Paper.schema3; -import java.util.logging.Level; - -import org.apache.parquet.Log; import org.apache.parquet.column.ParquetProperties; import org.apache.parquet.column.impl.ColumnWriteStoreV1; import org.apache.parquet.column.page.mem.MemPageStore; @@ -37,8 +34,6 @@ /** - * make sure {@link Log#LEVEL} is set to {@link Level#OFF} - * * @author Julien Le Dem * */ diff --git a/parquet-column/src/test/java/org/apache/parquet/io/TestColumnIO.java b/parquet-column/src/test/java/org/apache/parquet/io/TestColumnIO.java index e9e599affe..0aa342035f 100644 --- a/parquet-column/src/test/java/org/apache/parquet/io/TestColumnIO.java +++ b/parquet-column/src/test/java/org/apache/parquet/io/TestColumnIO.java @@ -44,7 +44,6 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import org.apache.parquet.Log; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.ColumnWriteStore; import org.apache.parquet.column.ColumnWriter; @@ -67,10 +66,12 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; import org.apache.parquet.schema.Type; import org.apache.parquet.schema.Type.Repetition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @RunWith(Parameterized.class) public class TestColumnIO { - private static final Log LOG = Log.getLog(TestColumnIO.class); + private static final Logger LOG = LoggerFactory.getLogger(TestColumnIO.class); private static final String oneOfEach = "message Document {\n" @@ -492,7 +493,7 @@ private RecordReaderImplementation getRecordReader(MessageColumnIO column } private void log(Object o) { - LOG.info(o); + LOG.info("{}", o); } private void validateFSA(int[][] expectedFSA, MessageColumnIO columnIO, RecordReaderImplementation recordReader) { diff --git a/parquet-common/src/main/java/org/apache/parquet/Closeables.java b/parquet-common/src/main/java/org/apache/parquet/Closeables.java index 2d8bb77469..086f6cc777 100644 --- a/parquet-common/src/main/java/org/apache/parquet/Closeables.java +++ b/parquet-common/src/main/java/org/apache/parquet/Closeables.java @@ -21,13 +21,16 @@ import java.io.Closeable; import java.io.IOException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * Utility for working with {@link java.io.Closeable}ss */ public final class Closeables { private Closeables() { } - private static final Log LOG = Log.getLog(Closeables.class); + private static final Logger LOG = LoggerFactory.getLogger(Closeables.class); /** * Closes a (potentially null) closeable. diff --git a/parquet-common/src/main/java/org/apache/parquet/bytes/BytesUtils.java b/parquet-common/src/main/java/org/apache/parquet/bytes/BytesUtils.java index 049f7bd24c..266685d845 100644 --- a/parquet-common/src/main/java/org/apache/parquet/bytes/BytesUtils.java +++ b/parquet-common/src/main/java/org/apache/parquet/bytes/BytesUtils.java @@ -25,7 +25,8 @@ import java.nio.ByteBuffer; import java.nio.charset.Charset; -import org.apache.parquet.Log; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * utility methods to deal with bytes @@ -34,7 +35,7 @@ * */ public class BytesUtils { - private static final Log LOG = Log.getLog(BytesUtils.class); + private static final Logger LOG = LoggerFactory.getLogger(BytesUtils.class); public static final Charset UTF8 = Charset.forName("UTF-8"); @@ -158,7 +159,7 @@ public static void writeIntLittleEndian(OutputStream out, int v) throws IOExcept out.write((v >>> 8) & 0xFF); out.write((v >>> 16) & 0xFF); out.write((v >>> 24) & 0xFF); - if (Log.DEBUG) LOG.debug("write le int: " + v + " => "+ ((v >>> 0) & 0xFF) + " " + ((v >>> 8) & 0xFF) + " " + ((v >>> 16) & 0xFF) + " " + ((v >>> 24) & 0xFF)); + if (LOG.isDebugEnabled()) LOG.debug("write le int: " + v + " => "+ ((v >>> 0) & 0xFF) + " " + ((v >>> 8) & 0xFF) + " " + ((v >>> 16) & 0xFF) + " " + ((v >>> 24) & 0xFF)); } /** diff --git a/parquet-common/src/test/java/org/apache/parquet/TestLog.java b/parquet-common/src/test/java/org/apache/parquet/TestLog.java deleted file mode 100644 index 4508b0d1f8..0000000000 --- a/parquet-common/src/test/java/org/apache/parquet/TestLog.java +++ /dev/null @@ -1,31 +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.parquet; - -import org.junit.Assert; -import org.junit.Test; - -public class TestLog { - - @Test - public void test() { - // Use a compile time log level of INFO for performance - Assert.assertFalse("Do not merge in log level DEBUG", Log.DEBUG); - } -} diff --git a/parquet-encoding/src/main/java/org/apache/parquet/bytes/BytesInput.java b/parquet-encoding/src/main/java/org/apache/parquet/bytes/BytesInput.java index cd9c6b2db5..6e593c2409 100644 --- a/parquet-encoding/src/main/java/org/apache/parquet/bytes/BytesInput.java +++ b/parquet-encoding/src/main/java/org/apache/parquet/bytes/BytesInput.java @@ -29,7 +29,8 @@ import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; -import org.apache.parquet.Log; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -44,7 +45,7 @@ * */ abstract public class BytesInput { - private static final Log LOG = Log.getLog(BytesInput.class); + private static final Logger LOG = LoggerFactory.getLogger(BytesInput.class); private static final boolean DEBUG = false;//Log.DEBUG; private static final EmptyBytesInput EMPTY_BYTES_INPUT = new EmptyBytesInput(); @@ -90,12 +91,12 @@ public static BytesInput from(ByteBuffer buffer, int offset, int length) { * @return a Bytes input that will write the given bytes */ public static BytesInput from(byte[] in) { - if (DEBUG) LOG.debug("BytesInput from array of " + in.length + " bytes"); + LOG.debug("BytesInput from array of {} bytes", in.length); return new ByteArrayBytesInput(in, 0 , in.length); } public static BytesInput from(byte[] in, int offset, int length) { - if (DEBUG) LOG.debug("BytesInput from array of " + length + " bytes"); + LOG.debug("BytesInput from array of {} bytes", length); return new ByteArrayBytesInput(in, offset, length); } @@ -189,7 +190,7 @@ public static BytesInput copy(BytesInput bytesInput) throws IOException { public byte[] toByteArray() throws IOException { BAOS baos = new BAOS((int)size()); this.writeAllTo(baos); - if (DEBUG) LOG.debug("converted " + size() + " to byteArray of " + baos.size() + " bytes"); + LOG.debug("converted {} to byteArray of {} bytes", size() , baos.size()); return baos.getBuf(); } @@ -228,7 +229,7 @@ public byte[] getBuf() { } private static class StreamBytesInput extends BytesInput { - private static final Log LOG = Log.getLog(BytesInput.StreamBytesInput.class); + private static final Logger LOG = LoggerFactory.getLogger(BytesInput.StreamBytesInput.class); private final InputStream in; private final int byteCount; @@ -240,13 +241,13 @@ private StreamBytesInput(InputStream in, int byteCount) { @Override public void writeAllTo(OutputStream out) throws IOException { - if (DEBUG) LOG.debug("write All "+ byteCount + " bytes"); + LOG.debug("write All {} bytes", byteCount); // TODO: more efficient out.write(this.toByteArray()); } public byte[] toByteArray() throws IOException { - if (DEBUG) LOG.debug("read all "+ byteCount + " bytes"); + LOG.debug("read all {} bytes", byteCount); byte[] buf = new byte[byteCount]; new DataInputStream(in).readFully(buf); return buf; @@ -260,7 +261,7 @@ public long size() { } private static class SequenceBytesIn extends BytesInput { - private static final Log LOG = Log.getLog(BytesInput.SequenceBytesIn.class); + private static final Logger LOG = LoggerFactory.getLogger(BytesInput.SequenceBytesIn.class); private final List inputs; private final long size; @@ -278,10 +279,11 @@ private SequenceBytesIn(List inputs) { @Override public void writeAllTo(OutputStream out) throws IOException { for (BytesInput input : inputs) { - if (DEBUG) LOG.debug("write " + input.size() + " bytes to out"); - if (DEBUG && input instanceof SequenceBytesIn) LOG.debug("{"); + + LOG.debug("write {} bytes to out", input.size()); + if (input instanceof SequenceBytesIn) LOG.debug("{"); input.writeAllTo(out); - if (DEBUG && input instanceof SequenceBytesIn) LOG.debug("}"); + if (input instanceof SequenceBytesIn) LOG.debug("}"); } } diff --git a/parquet-encoding/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java b/parquet-encoding/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java index 61555653d6..92674d4de6 100644 --- a/parquet-encoding/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java +++ b/parquet-encoding/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java @@ -30,9 +30,11 @@ import java.util.ArrayList; import java.util.List; -import org.apache.parquet.Log; import org.apache.parquet.OutputStreamCloseException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * Similar to a {@link ByteArrayOutputStream}, but uses a different strategy for growing that does not involve copying. * Where ByteArrayOutputStream is backed by a single array that "grows" by copying into a new larger array, this output @@ -54,7 +56,7 @@ * */ public class CapacityByteArrayOutputStream extends OutputStream { - private static final Log LOG = Log.getLog(CapacityByteArrayOutputStream.class); + private static final Logger LOG = LoggerFactory.getLogger(CapacityByteArrayOutputStream.class); private static final ByteBuffer EMPTY_SLAB = ByteBuffer.wrap(new byte[0]); private int initialSlabSize; @@ -167,11 +169,11 @@ private void addSlab(int minimumSize) { } if (nextSlabSize < minimumSize) { - if (Log.DEBUG) LOG.debug(format("slab size %,d too small for value of size %,d. Bumping up slab size", nextSlabSize, minimumSize)); + LOG.debug("slab size {} too small for value of size {}. Bumping up slab size", nextSlabSize, minimumSize); nextSlabSize = minimumSize; } - if (Log.DEBUG) LOG.debug(format("used %d slabs, adding new slab of size %d", slabs.size(), nextSlabSize)); + LOG.debug("used {} slabs, adding new slab of size {}", slabs.size(), nextSlabSize); this.currentSlab = allocator.allocate(nextSlabSize); this.slabs.add(currentSlab); @@ -265,7 +267,7 @@ public void reset() { // readjust slab size. // 7 = 2^3 - 1 so that doubling the initial size 3 times will get to the same size this.initialSlabSize = max(bytesUsed / 7, initialSlabSize); - if (Log.DEBUG) LOG.debug(String.format("initial slab of size %d", initialSlabSize)); + LOG.debug("initial slab of size {}", initialSlabSize); for (ByteBuffer slab : slabs) { allocator.release(slab); } diff --git a/parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBasedBitPackingEncoder.java b/parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBasedBitPackingEncoder.java index 448c0bec40..cc23e8f875 100644 --- a/parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBasedBitPackingEncoder.java +++ b/parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBasedBitPackingEncoder.java @@ -18,16 +18,16 @@ */ package org.apache.parquet.column.values.bitpacking; -import static org.apache.parquet.Log.DEBUG; -import static org.apache.parquet.bytes.BytesInput.concat; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.bytes.BytesUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.ArrayList; import java.util.List; -import org.apache.parquet.Log; -import org.apache.parquet.bytes.BytesInput; -import org.apache.parquet.bytes.BytesUtils; +import static org.apache.parquet.bytes.BytesInput.concat; /** * Uses the generated Byte based bit packing to write ints into a BytesInput @@ -36,7 +36,7 @@ * */ public class ByteBasedBitPackingEncoder { - private static final Log LOG = Log.getLog(ByteBasedBitPackingEncoder.class); + private static final Logger LOG = LoggerFactory.getLogger(ByteBasedBitPackingEncoder.class); private static final int VALUES_WRITTEN_AT_A_TIME = 8; @@ -99,7 +99,7 @@ private void initPackedSlab() { public BytesInput toBytes() throws IOException { int packedByteLength = packedPosition + BytesUtils.paddedByteCountFromBits(inputSize * bitWidth); - if (DEBUG) LOG.debug("writing " + (slabs.size() * slabSize + packedByteLength) + " bytes"); + LOG.debug("writing {} bytes", (slabs.size() * slabSize + packedByteLength)); if (inputSize > 0) { for (int i = inputSize; i < input.length; i++) { input[i] = 0; diff --git a/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPacking.java b/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPacking.java index ce9b3acb63..664fb1c0eb 100644 --- a/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPacking.java +++ b/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPacking.java @@ -27,12 +27,13 @@ import org.junit.Assert; import org.junit.Test; -import org.apache.parquet.Log; import org.apache.parquet.column.values.bitpacking.BitPacking.BitPackingReader; import org.apache.parquet.column.values.bitpacking.BitPacking.BitPackingWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TestBitPacking { - private static final Log LOG = Log.getLog(TestBitPacking.class); + private static final Logger LOG = LoggerFactory.getLogger(TestBitPacking.class); @Test public void testZero() throws IOException { @@ -170,7 +171,7 @@ private void validateEncodeDecode(int bitLength, int[] vals, String expected) w.finish(); byte[] bytes = baos.toByteArray(); LOG.debug("vals ("+bitLength+"): " + toString(vals)); - LOG.debug("bytes: " + toString(bytes)); + LOG.debug("bytes: {}", toString(bytes)); Assert.assertEquals(expected, toString(bytes)); ByteArrayInputStream bais = new ByteArrayInputStream(bytes); BitPackingReader r = BitPacking.createBitPackingReader(bitLength, bais, vals.length); @@ -178,7 +179,7 @@ private void validateEncodeDecode(int bitLength, int[] vals, String expected) for (int i = 0; i < result.length; i++) { result[i] = r.read(); } - LOG.debug("result: " + toString(result)); + LOG.debug("result: {}", toString(result)); assertArrayEquals(vals, result); } diff --git a/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestByteBitPacking.java b/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestByteBitPacking.java index b7dc26b29b..81467e66e7 100644 --- a/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestByteBitPacking.java +++ b/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestByteBitPacking.java @@ -26,23 +26,24 @@ import org.junit.Assert; import org.junit.Test; -import org.apache.parquet.Log; import org.apache.parquet.column.values.bitpacking.BitPacking.BitPackingReader; import org.apache.parquet.column.values.bitpacking.BitPacking.BitPackingWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TestByteBitPacking { - private static final Log LOG = Log.getLog(TestByteBitPacking.class); + private static final Logger LOG = LoggerFactory.getLogger(TestByteBitPacking.class); @Test public void testPackUnPack() { LOG.debug(""); LOG.debug("testPackUnPack"); for (int i = 1; i < 32; i++) { - LOG.debug("Width: " + i); + LOG.debug("Width: {}", i); int[] unpacked = new int[32]; int[] values = generateValues(i); packUnpack(Packer.BIG_ENDIAN.newBytePacker(i), values, unpacked); - LOG.debug("Output: " + TestBitPacking.toString(unpacked)); + LOG.debug("Output: {}", TestBitPacking.toString(unpacked)); Assert.assertArrayEquals("width "+i, values, unpacked); } } @@ -52,15 +53,15 @@ public void testPackUnPackLong() { LOG.debug(""); LOG.debug("testPackUnPackLong"); for (int i = 1; i < 64; i++) { - LOG.debug("Width: " + i); + LOG.debug("Width: {}", i); long[] unpacked32 = new long[32]; long[] unpacked8 = new long[32]; long[] values = generateValuesLong(i); packUnpack32(Packer.BIG_ENDIAN.newBytePackerForLong(i), values, unpacked32); - LOG.debug("Output 32: " + TestBitPacking.toString(unpacked32)); + LOG.debug("Output 32: {}", TestBitPacking.toString(unpacked32)); Assert.assertArrayEquals("width "+i, values, unpacked32); packUnpack8(Packer.BIG_ENDIAN.newBytePackerForLong(i), values, unpacked8); - LOG.debug("Output 8: " + TestBitPacking.toString(unpacked8)); + LOG.debug("Output 8: {}", TestBitPacking.toString(unpacked8)); Assert.assertArrayEquals("width "+i, values, unpacked8); } } @@ -68,14 +69,14 @@ public void testPackUnPackLong() { private void packUnpack(BytePacker packer, int[] values, int[] unpacked) { byte[] packed = new byte[packer.getBitWidth() * 4]; packer.pack32Values(values, 0, packed, 0); - LOG.debug("packed: " + TestBitPacking.toString(packed)); + LOG.debug("packed: {}", TestBitPacking.toString(packed)); packer.unpack32Values(ByteBuffer.wrap(packed), 0, unpacked, 0); } private void packUnpack32(BytePackerForLong packer, long[] values, long[] unpacked) { byte[] packed = new byte[packer.getBitWidth() * 4]; packer.pack32Values(values, 0, packed, 0); - LOG.debug("packed: " + TestBitPacking.toString(packed)); + LOG.debug("packed: {}", TestBitPacking.toString(packed)); packer.unpack32Values(packed, 0, unpacked, 0); } @@ -84,7 +85,7 @@ private void packUnpack8(BytePackerForLong packer, long[] values, long[] unpacke for (int i = 0; i < 4; i++) { packer.pack8Values(values, 8 * i, packed, packer.getBitWidth() * i); } - LOG.debug("packed: " + TestBitPacking.toString(packed)); + LOG.debug("packed: {}", TestBitPacking.toString(packed)); for (int i = 0; i < 4; i++) { packer.unpack8Values(packed, packer.getBitWidth() * i, unpacked, 8 * i); } @@ -95,7 +96,7 @@ private int[] generateValues(int bitWidth) { for (int j = 0; j < values.length; j++) { values[j] = (int)(Math.random() * 100000) % (int)Math.pow(2, bitWidth); } - LOG.debug("Input: " + TestBitPacking.toString(values)); + LOG.debug("Input: {}", TestBitPacking.toString(values)); return values; } @@ -105,7 +106,7 @@ private long[] generateValuesLong(int bitWidth) { for (int j = 0; j < values.length; j++) { values[j] = random.nextLong() & ((1l << bitWidth) - 1l); } - LOG.debug("Input: " + TestBitPacking.toString(values)); + LOG.debug("Input: {}", TestBitPacking.toString(values)); return values; } @@ -114,7 +115,7 @@ public void testPackUnPackAgainstHandWritten() throws IOException { LOG.debug(""); LOG.debug("testPackUnPackAgainstHandWritten"); for (int i = 1; i < 8; i++) { - LOG.debug("Width: " + i); + LOG.debug("Width: {}", i); byte[] packed = new byte[i * 4]; int[] unpacked = new int[32]; int[] values = generateValues(i); @@ -123,7 +124,7 @@ public void testPackUnPackAgainstHandWritten() throws IOException { final BytePacker packer = Packer.BIG_ENDIAN.newBytePacker(i); packer.pack32Values(values, 0, packed, 0); - LOG.debug("Generated: " + TestBitPacking.toString(packed)); + LOG.debug("Generated: {}", TestBitPacking.toString(packed)); // pack manual final ByteArrayOutputStream manualOut = new ByteArrayOutputStream(); @@ -132,7 +133,7 @@ public void testPackUnPackAgainstHandWritten() throws IOException { writer.write(values[j]); } final byte[] packedManualAsBytes = manualOut.toByteArray(); - LOG.debug("Manual: " + TestBitPacking.toString(packedManualAsBytes)); + LOG.debug("Manual: {}", TestBitPacking.toString(packedManualAsBytes)); // unpack manual final BitPackingReader reader = BitPacking.createBitPackingReader(i, new ByteArrayInputStream(packed), 32); @@ -140,7 +141,7 @@ public void testPackUnPackAgainstHandWritten() throws IOException { unpacked[j] = reader.read(); } - LOG.debug("Output: " + TestBitPacking.toString(unpacked)); + LOG.debug("Output: {}", TestBitPacking.toString(unpacked)); Assert.assertArrayEquals("width " + i, values, unpacked); } } @@ -149,9 +150,9 @@ public void testPackUnPackAgainstHandWritten() throws IOException { public void testPackUnPackAgainstLemire() throws IOException { for (Packer pack: Packer.values()) { LOG.debug(""); - LOG.debug("testPackUnPackAgainstLemire " + pack.name()); + LOG.debug("testPackUnPackAgainstLemire {}", pack.name()); for (int i = 1; i < 32; i++) { - LOG.debug("Width: " + i); + LOG.debug("Width: {}", i); int[] packed = new int[i]; int[] unpacked = new int[32]; int[] values = generateValues(i); @@ -178,17 +179,17 @@ public void testPackUnPackAgainstLemire() throws IOException { } } final byte[] packedByLemireAsBytes = lemireOut.toByteArray(); - LOG.debug("Lemire out: " + TestBitPacking.toString(packedByLemireAsBytes)); + LOG.debug("Lemire out: {}", TestBitPacking.toString(packedByLemireAsBytes)); // pack manual final BytePacker bytePacker = pack.newBytePacker(i); byte[] packedGenerated = new byte[i * 4]; bytePacker.pack32Values(values, 0, packedGenerated, 0); - LOG.debug("Gener. out: " + TestBitPacking.toString(packedGenerated)); + LOG.debug("Gener. out: {}", TestBitPacking.toString(packedGenerated)); Assert.assertEquals(pack.name() + " width " + i, TestBitPacking.toString(packedByLemireAsBytes), TestBitPacking.toString(packedGenerated)); bytePacker.unpack32Values(ByteBuffer.wrap(packedByLemireAsBytes), 0, unpacked, 0); - LOG.debug("Output: " + TestBitPacking.toString(unpacked)); + LOG.debug("Output: {}", TestBitPacking.toString(unpacked)); Assert.assertArrayEquals("width " + i, values, unpacked); } diff --git a/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestLemireBitPacking.java b/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestLemireBitPacking.java index 2c5fa58358..6a980da7f0 100644 --- a/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestLemireBitPacking.java +++ b/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestLemireBitPacking.java @@ -26,12 +26,13 @@ import org.junit.Assert; import org.junit.Test; -import org.apache.parquet.Log; import org.apache.parquet.column.values.bitpacking.BitPacking.BitPackingReader; import org.apache.parquet.column.values.bitpacking.BitPacking.BitPackingWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TestLemireBitPacking { - private static final Log LOG = Log.getLog(TestLemireBitPacking.class); + private static final Logger LOG = LoggerFactory.getLogger(TestLemireBitPacking.class); @Test public void testPackUnPack() { @@ -39,7 +40,7 @@ public void testPackUnPack() { LOG.debug(""); LOG.debug("testPackUnPack"); for (int i = 1; i < 32; i++) { - LOG.debug("Width: " + i); + LOG.debug("Width: {}", i); int[] values = generateValues(i); int[] unpacked = new int[32]; { @@ -73,7 +74,7 @@ private int[] generateValues(int bitWidth) { for (int j = 0; j < values.length; j++) { values[j] = (int)(Math.random() * 100000) % (int)Math.pow(2, bitWidth); } - LOG.debug("Input: " + TestBitPacking.toString(values)); + LOG.debug("Input: {}", TestBitPacking.toString(values)); return values; } @@ -82,7 +83,7 @@ public void testPackUnPackAgainstHandWritten() throws IOException { LOG.debug(""); LOG.debug("testPackUnPackAgainstHandWritten"); for (int i = 1; i < 8; i++) { - LOG.debug("Width: " + i); + LOG.debug("Width: {}", i); int[] packed = new int[i]; int[] unpacked = new int[32]; int[] values = generateValues(i); @@ -99,7 +100,7 @@ public void testPackUnPackAgainstHandWritten() throws IOException { lemireOut.write((v >>> 0) & 0xFF); } final byte[] packedByLemireAsBytes = lemireOut.toByteArray(); - LOG.debug("Lemire: " + TestBitPacking.toString(packedByLemireAsBytes)); + LOG.debug("Lemire: {}", TestBitPacking.toString(packedByLemireAsBytes)); // pack manual final ByteArrayOutputStream manualOut = new ByteArrayOutputStream(); @@ -108,7 +109,7 @@ public void testPackUnPackAgainstHandWritten() throws IOException { writer.write(values[j]); } final byte[] packedManualAsBytes = manualOut.toByteArray(); - LOG.debug("Manual: " + TestBitPacking.toString(packedManualAsBytes)); + LOG.debug("Manual: {}", TestBitPacking.toString(packedManualAsBytes)); // unpack manual final BitPackingReader reader = BitPacking.createBitPackingReader(i, new ByteArrayInputStream(packedByLemireAsBytes), 32); @@ -116,7 +117,7 @@ public void testPackUnPackAgainstHandWritten() throws IOException { unpacked[j] = reader.read(); } - LOG.debug("Output: " + TestBitPacking.toString(unpacked)); + LOG.debug("Output: {}", TestBitPacking.toString(unpacked)); Assert.assertArrayEquals("width " + i, values, unpacked); } } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java index bf99435ee9..91f30075b6 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java @@ -18,7 +18,6 @@ */ package org.apache.parquet.filter2.dictionarylevel; -import org.apache.parquet.Log; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.Dictionary; import org.apache.parquet.column.Encoding; @@ -30,6 +29,8 @@ import org.apache.parquet.filter2.predicate.UserDefinedPredicate; import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.HashMap; @@ -47,7 +48,7 @@ */ public class DictionaryFilter implements FilterPredicate.Visitor { - private static final Log LOG = Log.getLog(DictionaryFilter.class); + private static final Logger LOG = LoggerFactory.getLogger(DictionaryFilter.class); private static final boolean BLOCK_MIGHT_MATCH = false; private static final boolean BLOCK_CANNOT_MATCH = true; @@ -99,7 +100,7 @@ private > Set expandDictionary(ColumnChunkMetaData me case DOUBLE: dictSet.add(dict.decodeToDouble(i)); break; default: - LOG.warn("Unknown dictionary type" + meta.getType()); + LOG.warn("Unknown dictionary type{}", meta.getType()); } } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java b/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java index 6481b8f7c7..bf22b617b3 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java @@ -38,7 +38,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.parquet.CorruptStatistics; -import org.apache.parquet.Log; import org.apache.parquet.format.PageEncodingStats; import org.apache.parquet.hadoop.metadata.ColumnPath; import org.apache.parquet.format.ColumnChunk; @@ -71,6 +70,8 @@ import org.apache.parquet.schema.Type.Repetition; import org.apache.parquet.schema.TypeVisitor; import org.apache.parquet.schema.Types; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; // TODO: This file has become too long! // TODO: Lets split it up: https://issues.apache.org/jira/browse/PARQUET-310 @@ -80,7 +81,7 @@ public class ParquetMetadataConverter { public static final MetadataFilter SKIP_ROW_GROUPS = new SkipMetadataFilter(); public static final long MAX_STATS_SIZE = 4096; // limit stats to 4k - private static final Log LOG = Log.getLog(ParquetMetadataConverter.class); + private static final Logger LOG = LoggerFactory.getLogger(ParquetMetadataConverter.class); private final boolean useSignedStringMinMax; @@ -789,9 +790,9 @@ public FileMetaData visit(RangeMetadataFilter filter) throws IOException { return filterFileMetaDataByMidpoint(readFileMetaData(from), filter); } }); - if (Log.DEBUG) LOG.debug(fileMetaData); + LOG.debug("{}", fileMetaData); ParquetMetadata parquetMetadata = fromParquetMetadata(fileMetaData); - if (Log.DEBUG) LOG.debug(ParquetMetadata.toPrettyJSON(parquetMetadata)); + if (LOG.isDebugEnabled()) LOG.debug(ParquetMetadata.toPrettyJSON(parquetMetadata)); return parquetMetadata; } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageReadStore.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageReadStore.java index f428e8561b..f067679675 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageReadStore.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageReadStore.java @@ -25,7 +25,6 @@ import java.util.Map; import org.apache.parquet.Ints; -import org.apache.parquet.Log; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.page.DataPage; import org.apache.parquet.column.page.DataPageV1; @@ -36,6 +35,8 @@ import org.apache.parquet.column.page.PageReader; import org.apache.parquet.hadoop.CodecFactory.BytesDecompressor; import org.apache.parquet.io.ParquetDecodingException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * TODO: should this actually be called RowGroupImpl or something? @@ -44,7 +45,7 @@ * */ class ColumnChunkPageReadStore implements PageReadStore, DictionaryPageReadStore { - private static final Log LOG = Log.getLog(ColumnChunkPageReadStore.class); + private static final Logger LOG = LoggerFactory.getLogger(ColumnChunkPageReadStore.class); /** * PageReader for a single column chunk. A column chunk contains diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageWriteStore.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageWriteStore.java index 0fb9a18d93..ac3cd3b8b2 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageWriteStore.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageWriteStore.java @@ -18,7 +18,6 @@ */ package org.apache.parquet.hadoop; -import static org.apache.parquet.Log.INFO; import static org.apache.parquet.column.statistics.Statistics.getStatsBasedOnType; import java.io.ByteArrayOutputStream; @@ -30,7 +29,6 @@ import java.util.Map; import java.util.Set; -import org.apache.parquet.Log; import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.bytes.ConcatenatingByteArrayCollector; import org.apache.parquet.column.ColumnDescriptor; @@ -44,9 +42,11 @@ import org.apache.parquet.io.ParquetEncodingException; import org.apache.parquet.schema.MessageType; import org.apache.parquet.bytes.ByteBufferAllocator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class ColumnChunkPageWriteStore implements PageWriteStore { - private static final Log LOG = Log.getLog(ColumnChunkPageWriteStore.class); + private static final Logger LOG = LoggerFactory.getLogger(ColumnChunkPageWriteStore.class); private static ParquetMetadataConverter parquetMetadataConverter = new ParquetMetadataConverter(); @@ -191,8 +191,8 @@ public void writeToFileWriter(ParquetFileWriter writer) throws IOException { writer.writeDataPages(buf, uncompressedLength, compressedLength, totalStatistics, rlEncodings, dlEncodings, dataEncodings); writer.endColumn(); - if (INFO) { - LOG.info( + if (LOG.isDebugEnabled()) { + LOG.debug( String.format( "written %,dB for %s: %,d values, %,dB raw, %,dB comp, %d pages, encodings: %s", buf.size(), path, totalValueCount, uncompressedLength, compressedLength, pageCount, new HashSet(dataEncodings)) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/DirectCodecFactory.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/DirectCodecFactory.java index d90ab51093..344f3ec827 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/DirectCodecFactory.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/DirectCodecFactory.java @@ -33,12 +33,13 @@ import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.Compressor; import org.apache.hadoop.io.compress.Decompressor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.xerial.snappy.Snappy; import org.apache.parquet.bytes.ByteBufferAllocator; import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.hadoop.metadata.CompressionCodecName; -import org.apache.parquet.Log; import org.apache.parquet.ParquetRuntimeException; import org.apache.parquet.Preconditions; @@ -47,7 +48,7 @@ * direct memory, without requiring a copy into heap memory (where possible). */ class DirectCodecFactory extends CodecFactory implements AutoCloseable { - private static final Log LOG = Log.getLog(DirectCodecFactory.class); + private static final Logger LOG = LoggerFactory.getLogger(DirectCodecFactory.class); private final ByteBufferAllocator allocator; @@ -372,7 +373,7 @@ public Object makeObject() throws Exception { cPools.put(com.getClass(), compressorPool); compressorPool.returnObject(com); } else { - if (Log.DEBUG) { + if (LOG.isDebugEnabled()) { LOG.debug(String.format(BYTE_BUF_IMPL_NOT_FOUND_MSG, "compressor", codec.getClass().getName())); } } @@ -388,7 +389,7 @@ public Object makeObject() throws Exception { dePools.put(decom.getClass(), decompressorPool); decompressorPool.returnObject(decom); } else { - if (Log.DEBUG) { + if (LOG.isDebugEnabled()) { LOG.debug(String.format(BYTE_BUF_IMPL_NOT_FOUND_MSG, "decompressor", codec.getClass().getName())); } } @@ -408,7 +409,7 @@ public Object makeObject() throws Exception { } else { supportDirectDecompressor = false; - if (Log.DEBUG) { + if (LOG.isDebugEnabled()) { LOG.debug(String.format(BYTE_BUF_IMPL_NOT_FOUND_MSG, "compressor", codec.getClass().getName())); } } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordReader.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordReader.java index 85b669175a..88b3d2d12d 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordReader.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordReader.java @@ -1,4 +1,4 @@ -/* +/* * 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 @@ -6,9 +6,9 @@ * 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 @@ -27,7 +27,6 @@ import org.apache.hadoop.conf.Configuration; -import org.apache.parquet.Log; import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.filter.UnboundRecordFilter; import org.apache.parquet.filter2.compat.FilterCompat; @@ -42,16 +41,17 @@ import org.apache.parquet.io.api.RecordMaterializer; import org.apache.parquet.io.api.RecordMaterializer.RecordMaterializationException; import org.apache.parquet.schema.MessageType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static java.lang.String.format; -import static org.apache.parquet.Log.DEBUG; import static org.apache.parquet.Preconditions.checkNotNull; import static org.apache.parquet.hadoop.ParquetInputFormat.RECORD_FILTERING_ENABLED; import static org.apache.parquet.hadoop.ParquetInputFormat.RECORD_FILTERING_ENABLED_DEFAULT; import static org.apache.parquet.hadoop.ParquetInputFormat.STRICT_TYPE_CHECKING; class InternalParquetRecordReader { - private static final Log LOG = Log.getLog(InternalParquetRecordReader.class); + private static final Logger LOG = LoggerFactory.getLogger(InternalParquetRecordReader.class); private ColumnIOFactory columnIOFactory = null; private final Filter filter; @@ -110,7 +110,7 @@ private void checkRead() throws IOException { if (current == totalCountLoadedSoFar) { if (current != 0) { totalTimeSpentProcessingRecords += (System.currentTimeMillis() - startedAssemblingCurrentBlockAt); - if (Log.INFO) { + if (LOG.isInfoEnabled()) { LOG.info("Assembled and processed " + totalCountLoadedSoFar + " records from " + columnCount + " columns in " + totalTimeSpentProcessingRecords + " ms: "+((float)totalCountLoadedSoFar / totalTimeSpentProcessingRecords) + " rec/ms, " + ((float)totalCountLoadedSoFar * columnCount / totalTimeSpentProcessingRecords) + " cell/ms"); final long totalTime = totalTimeSpentProcessingRecords + totalTimeSpentReadingBytes; if (totalTime != 0) { @@ -130,8 +130,8 @@ private void checkRead() throws IOException { long timeSpentReading = System.currentTimeMillis() - t0; totalTimeSpentReadingBytes += timeSpentReading; BenchmarkCounter.incrementTime(timeSpentReading); - if (Log.INFO) LOG.info("block read in memory in " + timeSpentReading + " ms. row count = " + pages.getRowCount()); - if (Log.DEBUG) LOG.debug("initializing Record assembly with requested schema " + requestedSchema); + if (LOG.isInfoEnabled()) LOG.info("block read in memory in {} ms. row count = {}", timeSpentReading, pages.getRowCount()); + LOG.debug("initializing Record assembly with requested schema {}", requestedSchema); MessageColumnIO columnIO = columnIOFactory.getColumnIO(requestedSchema, fileSchema, strictTypeChecking); recordReader = columnIO.getRecordReader(pages, recordConverter, filterRecords ? filter : FilterCompat.NOOP); @@ -180,7 +180,7 @@ public void initialize(ParquetFileReader reader, Configuration configuration) this.filterRecords = configuration.getBoolean( RECORD_FILTERING_ENABLED, RECORD_FILTERING_ENABLED_DEFAULT); reader.setRequestedSchema(requestedSchema); - LOG.info("RecordReader initialized will read a total of " + total + " records."); + LOG.info("RecordReader initialized will read a total of {} records.", total); } public boolean nextKeyValue() throws IOException, InterruptedException { @@ -199,26 +199,26 @@ public boolean nextKeyValue() throws IOException, InterruptedException { } catch (RecordMaterializationException e) { // this might throw, but it's fatal if it does. unmaterializableRecordCounter.incErrors(e); - if (DEBUG) LOG.debug("skipping a corrupt record"); + LOG.debug("skipping a corrupt record"); continue; } if (recordReader.shouldSkipCurrentRecord()) { // this record is being filtered via the filter2 package - if (DEBUG) LOG.debug("skipping record"); + LOG.debug("skipping record"); continue; } if (currentValue == null) { // only happens with FilteredRecordReader at end of block current = totalCountLoadedSoFar; - if (DEBUG) LOG.debug("filtered record reader reached end of block"); + LOG.debug("filtered record reader reached end of block"); continue; } recordFound = true; - if (DEBUG) LOG.debug("read value: " + currentValue); + LOG.debug("read value: {}", currentValue); } catch (RuntimeException e) { throw new ParquetDecodingException(format("Can not read value at %d in block %d in file %s", current, currentBlock, reader.getPath()), e); } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java index 6c3dac5313..2a221ac85e 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java @@ -21,14 +21,12 @@ import static java.lang.Math.max; import static java.lang.Math.min; import static java.lang.String.format; -import static org.apache.parquet.Log.DEBUG; import static org.apache.parquet.Preconditions.checkNotNull; import java.io.IOException; import java.util.HashMap; import java.util.Map; -import org.apache.parquet.Log; import org.apache.parquet.column.ColumnWriteStore; import org.apache.parquet.column.ParquetProperties; import org.apache.parquet.hadoop.CodecFactory.BytesCompressor; @@ -38,9 +36,11 @@ import org.apache.parquet.io.MessageColumnIO; import org.apache.parquet.io.api.RecordConsumer; import org.apache.parquet.schema.MessageType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class InternalParquetRecordWriter { - private static final Log LOG = Log.getLog(InternalParquetRecordWriter.class); + private static final Logger LOG = LoggerFactory.getLogger(InternalParquetRecordWriter.class); private static final int MINIMUM_RECORD_COUNT_FOR_CHECK = 100; private static final int MAXIMUM_RECORD_COUNT_FOR_CHECK = 10000; @@ -139,7 +139,7 @@ private void checkBlockSizeReached() throws IOException { // flush the row group if it is within ~2 records of the limit // it is much better to be slightly under size than to be over at all if (memSize > (nextRowGroupSize - 2 * recordSize)) { - LOG.info(format("mem size %,d > %,d: flushing %,d records to disk.", memSize, nextRowGroupSize, recordCount)); + LOG.info("mem size {} > {}: flushing {} records to disk.", memSize, nextRowGroupSize, recordCount); flushRowGroupToStore(); initStore(); recordCountForNextMemCheck = min(max(MINIMUM_RECORD_COUNT_FOR_CHECK, recordCount / 2), MAXIMUM_RECORD_COUNT_FOR_CHECK); @@ -149,9 +149,7 @@ private void checkBlockSizeReached() throws IOException { max(MINIMUM_RECORD_COUNT_FOR_CHECK, (recordCount + (long)(nextRowGroupSize / ((float)recordSize))) / 2), // will check halfway recordCount + MAXIMUM_RECORD_COUNT_FOR_CHECK // will not look more than max records ahead ); - if (DEBUG) { - LOG.debug(format("Checked mem at %,d will check again at: %,d ", recordCount, recordCountForNextMemCheck)); - } + LOG.debug("Checked mem at {} will check again at: {}", recordCount, recordCountForNextMemCheck); } } } @@ -159,9 +157,9 @@ private void checkBlockSizeReached() throws IOException { private void flushRowGroupToStore() throws IOException { recordConsumer.flush(); - LOG.info(format("Flushing mem columnStore to file. allocated memory: %,d", columnStore.getAllocatedSize())); + LOG.info("Flushing mem columnStore to file. allocated memory: {}", columnStore.getAllocatedSize()); if (columnStore.getAllocatedSize() > (3 * rowGroupSizeThreshold)) { - LOG.warn("Too much memory used: " + columnStore.memUsageString()); + LOG.warn("Too much memory used: {}", columnStore.memUsageString()); } if (recordCount > 0) { diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/LruCache.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/LruCache.java index 44f9ecae71..c4e18b1667 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/LruCache.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/LruCache.java @@ -18,7 +18,8 @@ */ package org.apache.parquet.hadoop; -import org.apache.parquet.Log; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.LinkedHashMap; import java.util.Map; @@ -35,7 +36,7 @@ * so that the "staleness" of the value can be easily determined. */ final class LruCache> { - private static final Log LOG = Log.getLog(LruCache.class); + private static final Logger LOG = LoggerFactory.getLogger(LruCache.class); private static final float DEFAULT_LOAD_FACTOR = 0.75f; @@ -65,7 +66,7 @@ public LruCache(final int maxSize, final float loadFactor, final boolean accessO public boolean removeEldestEntry(final Map.Entry eldest) { boolean result = size() > maxSize; if (result) { - if (Log.DEBUG) { + if (LOG.isDebugEnabled()) { LOG.debug("Removing eldest entry in cache: " + eldest.getKey()); } @@ -84,9 +85,7 @@ public boolean removeEldestEntry(final Map.Entry eldest) { public V remove(final K key) { V oldValue = cacheMap.remove(key); if (oldValue != null) { - if (Log.DEBUG) { - LOG.debug("Removed cache entry for '" + key + "'"); - } + LOG.debug("Removed cache entry for '{}'", key); } return oldValue; } @@ -101,29 +100,29 @@ public V remove(final K key) { */ public void put(final K key, final V newValue) { if (newValue == null || !newValue.isCurrent(key)) { - if (Log.WARN) { - LOG.warn("Ignoring new cache entry for '" + key + "' because it is " - + (newValue == null ? "null" : "not current")); + if (LOG.isWarnEnabled()) { + LOG.warn("Ignoring new cache entry for '{}' because it is {}", key, + (newValue == null ? "null" : "not current")); } return; } V oldValue = cacheMap.get(key); if (oldValue != null && oldValue.isNewerThan(newValue)) { - if (Log.WARN) { - LOG.warn("Ignoring new cache entry for '" + key + "' because " - + "existing cache entry is newer"); + if (LOG.isWarnEnabled()) { + LOG.warn("Ignoring new cache entry for '{}' because " + + "existing cache entry is newer", key); } return; } // no existing value or new value is newer than old value oldValue = cacheMap.put(key, newValue); - if (Log.DEBUG) { + if (LOG.isDebugEnabled()) { if (oldValue == null) { - LOG.debug("Added new cache entry for '" + key + "'"); + LOG.debug("Added new cache entry for '{}'", key); } else { - LOG.debug("Overwrote existing cache entry for '" + key + "'"); + LOG.debug("Overwrote existing cache entry for '{}'", key); } } } @@ -145,10 +144,7 @@ public void clear() { */ public V getCurrentValue(final K key) { V value = cacheMap.get(key); - if (Log.DEBUG) { - LOG.debug("Value for '" + key + "' " + (value == null ? "not " : "") - + "in cache"); - } + LOG.debug("Value for '{}' {} in cache", key, (value == null ? "not " : "")); if (value != null && !value.isCurrent(key)) { // value is not current; remove it and return null remove(key); diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/MemoryManager.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/MemoryManager.java index 0c56bb29b2..dc5c31d477 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/MemoryManager.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/MemoryManager.java @@ -18,9 +18,10 @@ */ package org.apache.parquet.hadoop; -import org.apache.parquet.Log; import org.apache.parquet.ParquetRuntimeException; import org.apache.parquet.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.lang.management.ManagementFactory; import java.util.Collections; @@ -40,7 +41,7 @@ * When the sum exceeds, decrease each writer's allocation size by a ratio. */ public class MemoryManager { - private static final Log LOG = Log.getLog(MemoryManager.class); + private static final Logger LOG = LoggerFactory.getLogger(MemoryManager.class); static final float DEFAULT_MEMORY_POOL_RATIO = 0.95f; static final long DEFAULT_MIN_MEMORY_ALLOCATION = 1 * 1024 * 1024; // 1MB private final float memoryPoolRatio; @@ -59,7 +60,7 @@ public MemoryManager(float ratio, long minAllocation) { minMemoryAllocation = minAllocation; totalMemoryPool = Math.round((double) ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax () * ratio); - LOG.debug(String.format("Allocated total memory pool is: %,d", totalMemoryPool)); + LOG.debug("Allocated total memory pool is: {}", totalMemoryPool); } private void checkRatio(float ratio) { diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java index 4af26d0a0e..7b7534cbe4 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java @@ -18,7 +18,6 @@ */ package org.apache.parquet.hadoop; -import static org.apache.parquet.Log.DEBUG; import static org.apache.parquet.bytes.BytesUtils.readIntLittleEndian; import static org.apache.parquet.filter2.compat.RowGroupFilter.FilterLevel.DICTIONARY; import static org.apache.parquet.filter2.compat.RowGroupFilter.FilterLevel.STATISTICS; @@ -66,7 +65,6 @@ import org.apache.parquet.filter2.compat.FilterCompat; import org.apache.parquet.filter2.compat.RowGroupFilter; -import org.apache.parquet.Log; import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.page.DataPage; @@ -97,6 +95,8 @@ import org.apache.parquet.io.InputFile; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Internal implementation of the Parquet file reader as a block container @@ -106,7 +106,7 @@ */ public class ParquetFileReader implements Closeable { - private static final Log LOG = Log.getLog(ParquetFileReader.class); + private static final Logger LOG = LoggerFactory.getLogger(ParquetFileReader.class); public static String PARQUET_READ_PARALLELISM = "parquet.metadata.read.parallelism"; @@ -204,9 +204,7 @@ public Map call() throws Exception { if (toRead.size() > 0) { // read the footers of the files that did not have a summary file - if (Log.INFO) { - LOG.info("reading another " + toRead.size() + " footers"); - } + LOG.info("reading another {} footers", toRead.size()); result.addAll(readAllFootersInParallel(configuration, toRead, skipRowGroups)); } @@ -214,7 +212,7 @@ public Map call() throws Exception { } private static List runAllInParallel(int parallelism, List> toRun) throws ExecutionException { - LOG.info("Initiating action with parallelism: " + parallelism); + LOG.info("Initiating action with parallelism: {}", parallelism); ExecutorService threadPool = Executors.newFixedThreadPool(parallelism); try { List> futures = new ArrayList>(); @@ -360,12 +358,10 @@ static ParquetMetadata readSummaryMetadata(Configuration configuration, Path bas FileSystem fileSystem = basePath.getFileSystem(configuration); if (skipRowGroups && fileSystem.exists(commonMetaDataFile)) { // reading the summary file that does not contain the row groups - if (Log.INFO) LOG.info("reading summary file: " + commonMetaDataFile); + LOG.info("reading summary file: {}", commonMetaDataFile); return readFooter(configuration, commonMetaDataFile, filter(skipRowGroups)); } else if (fileSystem.exists(metadataFile)) { - if (Log.INFO) { - LOG.info("reading summary file: " + metadataFile); - } + LOG.info("reading summary file: {}", metadataFile); return readFooter(configuration, metadataFile, filter(skipRowGroups)); } else { return null; @@ -472,17 +468,13 @@ public static final ParquetMetadata readFooter( * @throws IOException if an error occurs while reading the file */ private static final ParquetMetadata readFooter(ParquetMetadataConverter converter, long fileLen, String filePath, SeekableInputStream f, MetadataFilter filter) throws IOException { - if (Log.DEBUG) { - LOG.debug("File length " + fileLen); - } + LOG.debug("File length {}", fileLen); int FOOTER_LENGTH_SIZE = 4; if (fileLen < MAGIC.length + FOOTER_LENGTH_SIZE + MAGIC.length) { // MAGIC + data + footer + footerIndex + MAGIC throw new RuntimeException(filePath + " is not a Parquet file (too small)"); } long footerLengthIndex = fileLen - FOOTER_LENGTH_SIZE - MAGIC.length; - if (Log.DEBUG) { - LOG.debug("reading footer index at " + footerLengthIndex); - } + LOG.debug("reading footer index at {}", footerLengthIndex); f.seek(footerLengthIndex); int footerLength = readIntLittleEndian(f); @@ -492,9 +484,7 @@ private static final ParquetMetadata readFooter(ParquetMetadataConverter convert throw new RuntimeException(filePath + " is not a Parquet file. expected magic number at tail " + Arrays.toString(MAGIC) + " but found " + Arrays.toString(magic)); } long footerIndex = footerLengthIndex - footerLength; - if (Log.DEBUG) { - LOG.debug("read footer length: " + footerLength + ", footer index: " + footerIndex); - } + LOG.debug("read footer length: {}, footer index: {}", footerLength, footerIndex); if (footerIndex < MAGIC.length || footerIndex >= footerLengthIndex) { throw new RuntimeException("corrupted file: the footer index is not within the file"); } @@ -926,9 +916,7 @@ public ColumnChunkPageReader readAllPages() throws IOException { valuesCountReadSoFar += dataHeaderV2.getNum_values(); break; default: - if (DEBUG) { - LOG.debug("skipping page of type " + pageHeader.getType() + " of size " + compressedPageSize); - } + LOG.debug("skipping page of type {} of size {}", pageHeader.getType(), compressedPageSize); this.skip(compressedPageSize); break; } @@ -1013,7 +1001,7 @@ public BytesInput readAsBytesInput(int size) throws IOException { // usually 13 to 19 bytes are missing int l1 = initPos + count - pos(); int l2 = size - l1; - LOG.info("completed the column chunk with " + l2 + " bytes"); + LOG.info("completed the column chunk with {} bytes", l2); return BytesInput.concat(super.readAsBytesInput(l1), BytesInput.copy(BytesInput.from(f, l2))); } return super.readAsBytesInput(size); diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java index f0fa7f5ace..57500bf440 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java @@ -18,7 +18,6 @@ */ package org.apache.parquet.hadoop; -import static org.apache.parquet.Log.DEBUG; import static org.apache.parquet.format.Util.writeFileMetaData; import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE; import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT; @@ -41,7 +40,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.parquet.Log; import org.apache.parquet.Preconditions; import org.apache.parquet.Strings; import org.apache.parquet.Version; @@ -67,6 +65,8 @@ import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; import org.apache.parquet.schema.TypeUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Internal implementation of the Parquet file writer as a block container @@ -75,7 +75,7 @@ * */ public class ParquetFileWriter { - private static final Log LOG = Log.getLog(ParquetFileWriter.class); + private static final Logger LOG = LoggerFactory.getLogger(ParquetFileWriter.class); private static ParquetMetadataConverter metadataConverter = new ParquetMetadataConverter(); @@ -274,7 +274,7 @@ public ParquetFileWriter(Configuration configuration, MessageType schema, */ public void start() throws IOException { state = state.start(); - if (DEBUG) LOG.debug(out.getPos() + ": start"); + LOG.debug("{}: start", out.getPos()); out.write(MAGIC); } @@ -285,7 +285,7 @@ public void start() throws IOException { */ public void startBlock(long recordCount) throws IOException { state = state.startBlock(); - if (DEBUG) LOG.debug(out.getPos() + ": start block"); + LOG.debug("{}: start block", out.getPos()); // out.write(MAGIC); // TODO: add a magic delimiter alignment.alignForRowGroup(out); @@ -325,7 +325,7 @@ public void startColumn(ColumnDescriptor descriptor, */ public void writeDictionaryPage(DictionaryPage dictionaryPage) throws IOException { state = state.write(); - if (DEBUG) LOG.debug(out.getPos() + ": write dictionary page: " + dictionaryPage.getDictionarySize() + " values"); + LOG.debug("{}: write dictionary page: {} values", out.getPos(), dictionaryPage.getDictionarySize()); currentChunkDictionaryPageOffset = out.getPos(); int uncompressedSize = dictionaryPage.getUncompressedSize(); int compressedPageSize = (int)dictionaryPage.getBytes().size(); // TODO: fix casts @@ -338,7 +338,7 @@ public void writeDictionaryPage(DictionaryPage dictionaryPage) throws IOExceptio long headerSize = out.getPos() - currentChunkDictionaryPageOffset; this.uncompressedLength += uncompressedSize + headerSize; this.compressedLength += compressedPageSize + headerSize; - if (DEBUG) LOG.debug(out.getPos() + ": write dictionary page content " + compressedPageSize); + LOG.debug("{}: write dictionary page content {}", out.getPos(), compressedPageSize); dictionaryPage.getBytes().writeAllTo(out); encodingStatsBuilder.addDictEncoding(dictionaryPage.getEncoding()); currentEncodings.add(dictionaryPage.getEncoding()); @@ -363,7 +363,7 @@ public void writeDataPage( Encoding valuesEncoding) throws IOException { state = state.write(); long beforeHeader = out.getPos(); - if (DEBUG) LOG.debug(beforeHeader + ": write data page: " + valueCount + " values"); + LOG.debug("{}: write data page: {} values", beforeHeader, valueCount); int compressedPageSize = (int)bytes.size(); metadataConverter.writeDataPageHeader( uncompressedPageSize, compressedPageSize, @@ -375,7 +375,7 @@ public void writeDataPage( long headerSize = out.getPos() - beforeHeader; this.uncompressedLength += uncompressedPageSize + headerSize; this.compressedLength += compressedPageSize + headerSize; - if (DEBUG) LOG.debug(out.getPos() + ": write data page content " + compressedPageSize); + LOG.debug("{}: write data page content {}", out.getPos(), compressedPageSize); bytes.writeAllTo(out); encodingStatsBuilder.addDataEncoding(valuesEncoding); currentEncodings.add(rlEncoding); @@ -401,7 +401,7 @@ public void writeDataPage( Encoding valuesEncoding) throws IOException { state = state.write(); long beforeHeader = out.getPos(); - if (DEBUG) LOG.debug(beforeHeader + ": write data page: " + valueCount + " values"); + LOG.debug("{}: write data page: {} values", beforeHeader, valueCount); int compressedPageSize = (int)bytes.size(); metadataConverter.writeDataPageHeader( uncompressedPageSize, compressedPageSize, @@ -414,7 +414,7 @@ public void writeDataPage( long headerSize = out.getPos() - beforeHeader; this.uncompressedLength += uncompressedPageSize + headerSize; this.compressedLength += compressedPageSize + headerSize; - if (DEBUG) LOG.debug(out.getPos() + ": write data page content " + compressedPageSize); + LOG.debug("{}: write data page content {}", out.getPos(), compressedPageSize); bytes.writeAllTo(out); currentStatistics.mergeStatistics(statistics); encodingStatsBuilder.addDataEncoding(valuesEncoding); @@ -438,11 +438,11 @@ void writeDataPages(BytesInput bytes, Set dlEncodings, List dataEncodings) throws IOException { state = state.write(); - if (DEBUG) LOG.debug(out.getPos() + ": write data pages"); + LOG.debug("{}: write data pages", out.getPos()); long headersSize = bytes.size() - compressedTotalPageSize; this.uncompressedLength += uncompressedTotalPageSize + headersSize; this.compressedLength += compressedTotalPageSize + headersSize; - if (DEBUG) LOG.debug(out.getPos() + ": write data pages content"); + LOG.debug("{}: write data pages content", out.getPos()); bytes.writeAllTo(out); encodingStatsBuilder.addDataEncodings(dataEncodings); if (rlEncodings.isEmpty()) { @@ -460,7 +460,7 @@ void writeDataPages(BytesInput bytes, */ public void endColumn() throws IOException { state = state.endColumn(); - if (DEBUG) LOG.debug(out.getPos() + ": end column"); + LOG.debug("{}: end column", out.getPos()); currentBlock.addColumn(ColumnChunkMetaData.get( currentChunkPath, currentChunkType, @@ -484,7 +484,7 @@ public void endColumn() throws IOException { */ public void endBlock() throws IOException { state = state.endBlock(); - if (DEBUG) LOG.debug(out.getPos() + ": end block"); + LOG.debug("{}: end block", out.getPos()); currentBlock.setRowCount(currentRecordCount); blocks.add(currentBlock); currentBlock = null; @@ -611,8 +611,7 @@ protected byte[] initialValue() { */ private static void copy(SeekableInputStream from, FSDataOutputStream to, long start, long length) throws IOException{ - if (DEBUG) LOG.debug( - "Copying " + length + " bytes at " + start + " to " + to.getPos()); + LOG.debug("Copying {} bytes at {} to {}" ,length , start , to.getPos()); from.seek(start); long bytesCopied = 0; byte[] buffer = COPY_BUFFER.get(); @@ -637,7 +636,7 @@ private static void copy(SeekableInputStream from, FSDataOutputStream to, */ public void end(Map extraMetaData) throws IOException { state = state.end(); - if (DEBUG) LOG.debug(out.getPos() + ": end"); + LOG.debug("{}: end", out.getPos()); ParquetMetadata footer = new ParquetMetadata(new FileMetaData(schema, extraMetaData, Version.FULL_VERSION), blocks); serializeFooter(footer, out); out.close(); @@ -647,7 +646,7 @@ private static void serializeFooter(ParquetMetadata footer, FSDataOutputStream o long footerIndex = out.getPos(); org.apache.parquet.format.FileMetaData parquetMetadata = metadataConverter.toParquetMetadata(CURRENT_VERSION, footer); writeFileMetaData(parquetMetadata, out); - if (DEBUG) LOG.debug(out.getPos() + ": footer length = " + (out.getPos() - footerIndex)); + LOG.debug("{}: footer length = {}" , out.getPos(), (out.getPos() - footerIndex)); BytesUtils.writeIntLittleEndian(out, (int) (out.getPos() - footerIndex)); out.write(MAGIC); } @@ -905,9 +904,7 @@ public void alignForRowGroup(FSDataOutputStream out) throws IOException { long remaining = dfsBlockSize - (out.getPos() % dfsBlockSize); if (isPaddingNeeded(remaining)) { - if (DEBUG) LOG.debug("Adding " + remaining + " bytes of padding (" + - "row group size=" + rowGroupSize + "B, " + - "block size=" + dfsBlockSize + "B)"); + LOG.debug("Adding {} bytes of padding (row group size={}B, block size={}B)", remaining, rowGroupSize, dfsBlockSize); for (; remaining > 0; remaining -= zeros.length) { out.write(zeros, 0, (int) Math.min((long) zeros.length, remaining)); } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetInputFormat.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetInputFormat.java index 1fe57f9fc5..7c5b5beded 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetInputFormat.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetInputFormat.java @@ -51,7 +51,6 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.input.FileSplit; -import org.apache.parquet.Log; import org.apache.parquet.Preconditions; import org.apache.parquet.filter.UnboundRecordFilter; import org.apache.parquet.filter2.compat.FilterCompat; @@ -72,6 +71,8 @@ import org.apache.parquet.io.ParquetDecodingException; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.MessageTypeParser; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * The input format to read a Parquet file. @@ -93,7 +94,7 @@ */ public class ParquetInputFormat extends FileInputFormat { - private static final Log LOG = Log.getLog(ParquetInputFormat.class); + private static final Logger LOG = LoggerFactory.getLogger(ParquetInputFormat.class); /** * key to configure the ReadSupport implementation @@ -382,7 +383,7 @@ private static List getAllFileRecursively( result.add(file); } } - LOG.info("Total input paths to process : " + result.size()); + LOG.info("Total input paths to process : {}", result.size()); return result; } @@ -424,7 +425,7 @@ public List