http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-column/src/main/java/org/apache/parquet/io/RecordConsumerLoggingWrapper.java
----------------------------------------------------------------------
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 7a8b1c1..b90e216 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 @@ import org.apache.parquet.io.api.RecordConsumer;
*
*/
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 class RecordConsumerLoggingWrapper extends RecordConsumer {
*/
@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 @@ public class RecordConsumerLoggingWrapper extends RecordConsumer {
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 @@ public class RecordConsumerLoggingWrapper extends RecordConsumer {
*/
@Override
public void startGroup() {
- if (DEBUG) ++indent;
- if (DEBUG) log("<!-- start group -->");
+ ++indent;
+ log("<!-- start group -->");
delegate.startGroup();
}
@@ -85,7 +88,7 @@ public class RecordConsumerLoggingWrapper extends RecordConsumer {
*/
@Override
public void addInteger(int value) {
- if (DEBUG) log(value);
+ log(value);
delegate.addInteger(value);
}
@@ -94,7 +97,7 @@ public class RecordConsumerLoggingWrapper extends RecordConsumer {
*/
@Override
public void addLong(long value) {
- if (DEBUG) log(value);
+ log(value);
delegate.addLong(value);
}
@@ -103,7 +106,7 @@ public class RecordConsumerLoggingWrapper extends RecordConsumer {
*/
@Override
public void addBoolean(boolean value) {
- if (DEBUG) log(value);
+ log(value);
delegate.addBoolean(value);
}
@@ -112,7 +115,7 @@ public class RecordConsumerLoggingWrapper extends RecordConsumer {
*/
@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 class RecordConsumerLoggingWrapper extends RecordConsumer {
*/
@Override
public void addFloat(float value) {
- if (DEBUG) log(value);
+ log(value);
delegate.addFloat(value);
}
@@ -130,7 +133,7 @@ public class RecordConsumerLoggingWrapper extends RecordConsumer {
*/
@Override
public void addDouble(double value) {
- if (DEBUG) log(value);
+ log(value);
delegate.addDouble(value);
}
@@ -139,7 +142,7 @@ public class RecordConsumerLoggingWrapper extends RecordConsumer {
*/
@Override
public void flush() {
- if (DEBUG) log("<!-- flush -->");
+ log("<!-- flush -->");
delegate.flush();
}
@@ -148,8 +151,8 @@ public class RecordConsumerLoggingWrapper extends RecordConsumer {
*/
@Override
public void endGroup() {
- if (DEBUG) log("<!-- end group -->");
- if (DEBUG) --indent;
+ log("<!-- end group -->");
+ --indent;
delegate.endGroup();
}
@@ -158,12 +161,12 @@ public class RecordConsumerLoggingWrapper extends RecordConsumer {
*/
@Override
public void endField(String field, int index) {
- if (DEBUG) logClose(field);
+ logClose(field);
delegate.endField(field, index);
}
private void logClose(String field) {
- log("</"+field+">");
+ log("</{}>", field);
}
/**
@@ -171,7 +174,7 @@ public class RecordConsumerLoggingWrapper extends RecordConsumer {
*/
@Override
public void startMessage() {
- if (DEBUG) log("<!-- start message -->");
+ log("<!-- start message -->");
delegate.startMessage();
}
@@ -181,7 +184,7 @@ public class RecordConsumerLoggingWrapper extends RecordConsumer {
@Override
public void endMessage() {
delegate.endMessage();
- if (DEBUG) log("<!-- end message -->");
+ log("<!-- end message -->");
}
}
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-column/src/main/java/org/apache/parquet/io/RecordReaderImplementation.java
----------------------------------------------------------------------
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 7a87cbb..af7d4a5 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.HashMap;
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.RecordConsumer;
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 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
* @param <T> the type of the materialized record
*/
class RecordReaderImplementation<T> extends RecordReader<T> {
- 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 @@ class RecordReaderImplementation<T> extends RecordReader<T> {
}
private RecordConsumer wrap(RecordConsumer recordConsumer) {
- if (Log.DEBUG) {
+ if (LOG.isDebugEnabled()) {
return new RecordConsumerLoggingWrapper(recordConsumer);
}
return recordConsumer;
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-column/src/main/java/org/apache/parquet/io/ValidatingRecordConsumer.java
----------------------------------------------------------------------
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 46f0aae..c27381a 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.ArrayDeque;
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 @@ import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.*;
*
*/
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 @@ public class ValidatingRecordConsumer extends RecordConsumer {
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 @@ public class ValidatingRecordConsumer extends RecordConsumer {
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:
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-column/src/main/java/org/apache/parquet/schema/MessageTypeParser.java
----------------------------------------------------------------------
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 b7274c2..f0c178a 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.Arrays;
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 @@ import org.apache.parquet.schema.Types.PrimitiveBuilder;
* @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 {
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-column/src/test/java/org/apache/parquet/column/mem/TestMemColumn.java
----------------------------------------------------------------------
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 42c1776..c855339 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 static org.junit.Assert.assertEquals;
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.example.DummyRecordConverter;
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 class TestMemColumn {
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 class TestMemColumn {
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) {
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageReader.java
----------------------------------------------------------------------
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 a6e8910..5373c9a 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<DataPage> pages;
@@ -54,7 +54,7 @@ public class MemPageReader implements PageReader {
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");
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageStore.java
----------------------------------------------------------------------
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 219e5cd..cdde894 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.PageReadStore;
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<ColumnDescriptor, MemPageWriter> pageWriters = new HashMap<ColumnDescriptor, MemPageWriter>();
@@ -62,7 +63,7 @@ public class MemPageStore implements PageReadStore, PageWriteStore {
throw new UnknownColumnException(descriptor);
}
List<DataPage> pages = new ArrayList<DataPage>(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());
}
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageWriter.java
----------------------------------------------------------------------
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 d5bfe22..51e1da4 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<DataPage> pages = new ArrayList<DataPage>();
private DictionaryPage dictionaryPage;
@@ -53,7 +53,7 @@ public class MemPageWriter implements PageWriter {
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 class MemPageWriter implements PageWriter {
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 class MemPageWriter implements PageWriter {
}
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
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPackingColumn.java
----------------------------------------------------------------------
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 2f311ec..c777651 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
@@ -26,12 +26,13 @@ import java.io.IOException;
import org.junit.Test;
-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 {
@@ -161,7 +162,7 @@ public class TestBitPackingColumn {
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) {
@@ -169,7 +170,7 @@ public class TestBitPackingColumn {
}
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, bytes, 0);
@@ -177,7 +178,7 @@ public class TestBitPackingColumn {
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);
}
}
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-column/src/test/java/org/apache/parquet/io/PerfTest.java
----------------------------------------------------------------------
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 292ed83..bf783df 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,10 +24,6 @@ import static org.apache.parquet.example.Paper.schema;
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.WriterVersion;
import org.apache.parquet.column.ParquetProperties;
import org.apache.parquet.column.impl.ColumnWriteStoreV1;
import org.apache.parquet.column.page.mem.MemPageStore;
@@ -38,8 +34,6 @@ import org.apache.parquet.schema.MessageType;
/**
- * make sure {@link Log#LEVEL} is set to {@link Level#OFF}
- *
* @author Julien Le Dem
*
*/
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-column/src/test/java/org/apache/parquet/io/TestColumnIO.java
----------------------------------------------------------------------
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 3d38fe6..8cf3975 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.Test;
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;
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 @@ public class TestColumnIO {
}
private void log(Object o) {
- LOG.info(o);
+ LOG.info("{}", o);
}
private void validateFSA(int[][] expectedFSA, MessageColumnIO columnIO, RecordReaderImplementation<?> recordReader) {
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-common/src/main/java/org/apache/parquet/Closeables.java
----------------------------------------------------------------------
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 2d8bb77..086f6cc 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 @@ package org.apache.parquet;
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.
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-common/src/main/java/org/apache/parquet/bytes/BytesUtils.java
----------------------------------------------------------------------
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 d96a1e5..0ef3862 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
@@ -24,7 +24,8 @@ import java.io.InputStream;
import java.io.OutputStream;
import java.nio.charset.Charset;
-import org.apache.parquet.Log;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
* utility methods to deal with bytes
@@ -33,7 +34,7 @@ import org.apache.parquet.Log;
*
*/
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");
@@ -142,7 +143,7 @@ public class BytesUtils {
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));
}
/**
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-common/src/test/java/org/apache/parquet/TestLog.java
----------------------------------------------------------------------
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 4508b0d..0000000
--- 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);
- }
-}
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-encoding/src/main/java/org/apache/parquet/bytes/BytesInput.java
----------------------------------------------------------------------
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 ac334ae..eb62958 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
@@ -26,7 +26,8 @@ import java.io.OutputStream;
import java.util.Arrays;
import java.util.List;
-import org.apache.parquet.Log;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
@@ -41,7 +42,7 @@ import org.apache.parquet.Log;
*
*/
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();
@@ -78,12 +79,12 @@ abstract public class BytesInput {
* @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);
}
@@ -160,7 +161,7 @@ abstract public class BytesInput {
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();
}
@@ -181,7 +182,7 @@ abstract public class BytesInput {
}
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;
@@ -193,13 +194,13 @@ abstract public class BytesInput {
@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;
@@ -213,7 +214,7 @@ abstract public class BytesInput {
}
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<BytesInput> inputs;
private final long size;
@@ -231,10 +232,11 @@ abstract public class BytesInput {
@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("}");
}
}
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-encoding/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java
----------------------------------------------------------------------
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 1670f9c..9eb1909 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
@@ -31,6 +31,8 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.parquet.Log;
+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.
@@ -53,7 +55,7 @@ import org.apache.parquet.Log;
*
*/
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 byte[] EMPTY_SLAB = new byte[0];
private int initialSlabSize;
@@ -139,11 +141,11 @@ public class CapacityByteArrayOutputStream extends OutputStream {
}
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 = new byte[nextSlabSize];
this.slabs.add(currentSlab);
@@ -221,7 +223,7 @@ public class CapacityByteArrayOutputStream extends OutputStream {
// 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);
this.slabs.clear();
this.bytesAllocated = 0;
this.bytesUsed = 0;
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBasedBitPackingEncoder.java
----------------------------------------------------------------------
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 448c0be..cc23e8f 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 @@ import org.apache.parquet.bytes.BytesUtils;
*
*/
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 @@ public class ByteBasedBitPackingEncoder {
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;
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPacking.java
----------------------------------------------------------------------
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 64679e5..f42b136 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 java.io.IOException;
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 @@ public class TestBitPacking {
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 @@ public class TestBitPacking {
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);
}
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestByteBitPacking.java
----------------------------------------------------------------------
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 a5ce37e..2ba2012 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
@@ -24,24 +24,24 @@ import java.io.IOException;
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);
}
}
@@ -49,7 +49,7 @@ public class TestByteBitPacking {
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(packed, 0, unpacked, 0);
}
@@ -58,7 +58,7 @@ public class TestByteBitPacking {
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;
}
@@ -67,7 +67,7 @@ public class TestByteBitPacking {
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);
@@ -76,7 +76,7 @@ public class TestByteBitPacking {
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();
@@ -85,7 +85,7 @@ public class TestByteBitPacking {
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);
@@ -93,7 +93,7 @@ public class TestByteBitPacking {
unpacked[j] = reader.read();
}
- LOG.debug("Output: " + TestBitPacking.toString(unpacked));
+ LOG.debug("Output: {}", TestBitPacking.toString(unpacked));
Assert.assertArrayEquals("width " + i, values, unpacked);
}
}
@@ -102,9 +102,9 @@ public class TestByteBitPacking {
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);
@@ -131,17 +131,17 @@ public class TestByteBitPacking {
}
}
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(packedByLemireAsBytes, 0, unpacked, 0);
- LOG.debug("Output: " + TestBitPacking.toString(unpacked));
+ LOG.debug("Output: {}", TestBitPacking.toString(unpacked));
Assert.assertArrayEquals("width " + i, values, unpacked);
}
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestLemireBitPacking.java
----------------------------------------------------------------------
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 e0c97e0..c3815b0 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
@@ -25,12 +25,13 @@ import java.io.IOException;
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() {
@@ -38,7 +39,7 @@ public class TestLemireBitPacking {
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];
{
@@ -72,7 +73,7 @@ public class TestLemireBitPacking {
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;
}
@@ -81,7 +82,7 @@ public class TestLemireBitPacking {
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);
@@ -98,7 +99,7 @@ public class TestLemireBitPacking {
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();
@@ -107,7 +108,7 @@ public class TestLemireBitPacking {
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);
@@ -115,7 +116,7 @@ public class TestLemireBitPacking {
unpacked[j] = reader.read();
}
- LOG.debug("Output: " + TestBitPacking.toString(unpacked));
+ LOG.debug("Output: {}", TestBitPacking.toString(unpacked));
Assert.assertArrayEquals("width " + i, values, unpacked);
}
}
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java
----------------------------------------------------------------------
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 bf99435..91f3007 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.Operators.*;
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 @@ import static org.apache.parquet.Preconditions.checkNotNull;
*/
public class DictionaryFilter implements FilterPredicate.Visitor<Boolean> {
- 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 @@ public class DictionaryFilter implements FilterPredicate.Visitor<Boolean> {
case DOUBLE: dictSet.add(dict.decodeToDouble(i));
break;
default:
- LOG.warn("Unknown dictionary type" + meta.getType());
+ LOG.warn("Unknown dictionary type{}", meta.getType());
}
}
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java
----------------------------------------------------------------------
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 b5ddbc1..cc43008 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 java.util.concurrent.ConcurrentHashMap;
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.PrimitiveType.PrimitiveTypeName;
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 class ParquetMetadataConverter {
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;
}
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageReadStore.java
----------------------------------------------------------------------
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 2e8f84a..c6eb3db 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.List;
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.PageReadStore;
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 @@ import org.apache.parquet.io.ParquetDecodingException;
*
*/
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
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageWriteStore.java
----------------------------------------------------------------------
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 7f20f52..f4925bc 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.List;
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;
@@ -43,9 +41,11 @@ import org.apache.parquet.format.converter.ParquetMetadataConverter;
import org.apache.parquet.hadoop.CodecFactory.BytesCompressor;
import org.apache.parquet.io.ParquetEncodingException;
import org.apache.parquet.schema.MessageType;
+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();
@@ -186,8 +186,8 @@ class ColumnChunkPageWriteStore implements PageWriteStore {
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<Encoding>(dataEncodings))
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordReader.java
----------------------------------------------------------------------
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 85b6691..88b3d2d 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 java.util.Set;
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.ParquetDecodingException;
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<T> {
- 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 @@ class InternalParquetRecordReader<T> {
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 @@ class InternalParquetRecordReader<T> {
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 @@ class InternalParquetRecordReader<T> {
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 @@ class InternalParquetRecordReader<T> {
} 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);
}
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java
----------------------------------------------------------------------
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 d5c57ab..44a409d 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 @@ package org.apache.parquet.hadoop;
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.ColumnIOFactory;
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<T> {
- 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 @@ class InternalParquetRecordWriter<T> {
// 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,7 +149,7 @@ class InternalParquetRecordWriter<T> {
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);
}
}
}
@@ -157,9 +157,9 @@ class InternalParquetRecordWriter<T> {
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) {
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/LruCache.java
----------------------------------------------------------------------
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 44f9eca..c4e18b1 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 @@ import java.util.Map;
* so that the "staleness" of the value can be easily determined.
*/
final class LruCache<K, V extends LruCache.Value<K, V>> {
- 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 @@ final class LruCache<K, V extends LruCache.Value<K, V>> {
public boolean removeEldestEntry(final Map.Entry<K, V> 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 @@ final class LruCache<K, V extends LruCache.Value<K, V>> {
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 @@ final class LruCache<K, V extends LruCache.Value<K, V>> {
*/
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 @@ final class LruCache<K, V extends LruCache.Value<K, V>> {
*/
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);
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/MemoryManager.java
----------------------------------------------------------------------
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 0c56bb2..dc5c31d 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 @@ import java.util.Map;
* 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 class MemoryManager {
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) {
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8e2009b8/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java
----------------------------------------------------------------------
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 9f8eb07..b41bac1 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;
@@ -63,7 +62,6 @@ import org.apache.parquet.column.page.DictionaryPageReadStore;
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;
@@ -94,6 +92,8 @@ import org.apache.parquet.io.ParquetDecodingException;
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
@@ -103,7 +103,7 @@ import org.apache.parquet.schema.PrimitiveType;
*/
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";
@@ -201,7 +201,7 @@ public class ParquetFileReader implements Closeable {
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));
}
@@ -209,7 +209,7 @@ public class ParquetFileReader implements Closeable {
}
private static <T> List<T> runAllInParallel(int parallelism, List<Callable<T>> toRun) throws ExecutionException {
- LOG.info("Initiating action with parallelism: " + parallelism);
+ LOG.info("Initiating action with parallelism: {}", parallelism);
ExecutorService threadPool = Executors.newFixedThreadPool(parallelism);
try {
List<Future<T>> futures = new ArrayList<Future<T>>();
@@ -346,10 +346,10 @@ public class ParquetFileReader implements Closeable {
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;
@@ -460,17 +460,13 @@ public class ParquetFileReader implements Closeable {
* @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);
@@ -480,9 +476,7 @@ public class ParquetFileReader implements Closeable {
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");
}
@@ -909,7 +903,7 @@ public class ParquetFileReader implements Closeable {
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;
}
@@ -993,7 +987,7 @@ public class ParquetFileReader implements Closeable {
// usually 13 to 19 bytes are missing
int l1 = 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);
|