parquet-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jul...@apache.org
Subject [4/4] parquet-mr git commit: PARQUET-423: Replace old Log class with SLF4J Logging
Date Wed, 26 Oct 2016 16:09:58 GMT
PARQUET-423: Replace old Log class with SLF4J Logging

And make writing files less noisy

Author: Niels Basjes <nbasjes@bol.com>

Closes #369 from nielsbasjes/PARQUET-423-2 and squashes the following commits:

b31e30f [Niels Basjes] Merge branch 'master' of github.com:apache/parquet-mr into PARQUET-423-2
2d4db4b [Niels Basjes] Merge branch 'PARQUET-423-2' of github.com:nielsbasjes/parquet-mr into PARQUET-423-2
49fcaa7 [Niels Basjes] PARQUET-423: Remove debug logging statements in high performance sections during build time
aaaf4a6 [Niels Basjes] Merge branch 'PARQUET-423-2' of github.com:nielsbasjes/parquet-mr into PARQUET-423-2
745666e [Niels Basjes] Undo needless change
94e0c7a [Niels Basjes] PARQUET-423: Further optimize logging performance
b72f924 [Niels Basjes] PARQUET-423: Improved the performance
cb7eb61 [Niels Basjes] PARQUET-423: Workaround AVRO errors
7d161b3 [Niels Basjes] PARQUET-423: Restore the old (obsolete) Log class
05d6a47 [Niels Basjes] PARQUET-423: Replace old Log class with SLF4J Logging
692ebfb [Niels Basjes] Undo needless change
f1ede3d [Niels Basjes] PARQUET-423: Further optimize logging performance
a0c6b59 [Niels Basjes] PARQUET-423: Improved the performance
67bef9b [Niels Basjes] PARQUET-423: Workaround AVRO errors
87cd64f [Niels Basjes] PARQUET-423: Restore the old (obsolete) Log class
96d97d5 [Niels Basjes] PARQUET-423: Replace old Log class with SLF4J Logging


Project: http://git-wip-us.apache.org/repos/asf/parquet-mr/repo
Commit: http://git-wip-us.apache.org/repos/asf/parquet-mr/commit/df9d8e41
Tree: http://git-wip-us.apache.org/repos/asf/parquet-mr/tree/df9d8e41
Diff: http://git-wip-us.apache.org/repos/asf/parquet-mr/diff/df9d8e41

Branch: refs/heads/master
Commit: df9d8e415436292ae33e1ca0b8da256640de9710
Parents: aa416b5
Author: Niels Basjes <nbasjes@bol.com>
Authored: Wed Oct 26 09:09:56 2016 -0700
Committer: Julien Le Dem <julien@dremio.com>
Committed: Wed Oct 26 09:09:56 2016 -0700

----------------------------------------------------------------------
 .../parquet/avro/TestInputOutputFormat.java     |  9 ++--
 .../avro/TestReflectInputOutputFormat.java      |  9 ++--
 .../avro/TestSpecificInputOutputFormat.java     |  9 ++--
 .../apache/parquet/CorruptDeltaByteArrays.java  | 11 ++--
 .../org/apache/parquet/CorruptStatistics.java   |  4 +-
 .../parquet/column/impl/ColumnReaderImpl.java   | 20 ++++----
 .../parquet/column/impl/ColumnWriterV1.java     | 14 ++---
 .../parquet/column/impl/ColumnWriterV2.java     | 16 +++---
 .../bitpacking/BitPackingValuesReader.java      |  7 +--
 .../bitpacking/ByteBitPackingValuesReader.java  |  7 +--
 .../DeltaLengthByteArrayValuesReader.java       |  8 +--
 .../DeltaLengthByteArrayValuesWriter.java       |  7 +--
 .../dictionary/DictionaryValuesReader.java      | 11 ++--
 .../dictionary/DictionaryValuesWriter.java      | 11 ++--
 .../values/plain/BinaryPlainValuesReader.java   |  8 +--
 .../values/plain/BooleanPlainValuesReader.java  |  8 +--
 .../FixedLenByteArrayPlainValuesReader.java     |  9 ++--
 .../FixedLenByteArrayPlainValuesWriter.java     |  7 +--
 .../column/values/plain/PlainValuesReader.java  |  9 ++--
 .../column/values/plain/PlainValuesWriter.java  |  7 +--
 .../rle/RunLengthBitPackingHybridDecoder.java   | 12 ++---
 .../rle/RunLengthBitPackingHybridEncoder.java   | 12 ++---
 .../org/apache/parquet/example/data/Group.java  | 10 ++--
 .../parquet/filter2/compat/FilterCompat.java    |  9 ++--
 .../org/apache/parquet/io/BaseRecordReader.java | 28 +++++-----
 .../java/org/apache/parquet/io/ColumnIO.java    |  3 --
 .../org/apache/parquet/io/GroupColumnIO.java    |  5 +-
 .../org/apache/parquet/io/MessageColumnIO.java  | 54 +++++++++++---------
 .../io/RecordConsumerLoggingWrapper.java        | 49 +++++++++---------
 .../parquet/io/RecordReaderImplementation.java  |  7 +--
 .../parquet/io/ValidatingRecordConsumer.java    | 10 ++--
 .../parquet/schema/MessageTypeParser.java       |  5 +-
 .../parquet/column/mem/TestMemColumn.java       |  9 ++--
 .../parquet/column/page/mem/MemPageReader.java  |  8 +--
 .../parquet/column/page/mem/MemPageStore.java   | 17 +++---
 .../parquet/column/page/mem/MemPageWriter.java  | 27 +++++-----
 .../values/bitpacking/TestBitPackingColumn.java | 11 ++--
 .../java/org/apache/parquet/io/PerfTest.java    |  5 --
 .../org/apache/parquet/io/TestColumnIO.java     |  7 +--
 .../java/org/apache/parquet/Closeables.java     |  5 +-
 .../org/apache/parquet/bytes/BytesUtils.java    |  7 +--
 .../test/java/org/apache/parquet/TestLog.java   | 31 -----------
 .../org/apache/parquet/bytes/BytesInput.java    | 26 +++++-----
 .../bytes/CapacityByteArrayOutputStream.java    | 12 +++--
 .../bitpacking/ByteBasedBitPackingEncoder.java  | 14 ++---
 .../values/bitpacking/TestBitPacking.java       |  9 ++--
 .../values/bitpacking/TestByteBitPacking.java   | 43 ++++++++--------
 .../values/bitpacking/TestLemireBitPacking.java | 17 +++---
 .../dictionarylevel/DictionaryFilter.java       |  7 +--
 .../converter/ParquetMetadataConverter.java     |  9 ++--
 .../hadoop/ColumnChunkPageReadStore.java        |  5 +-
 .../hadoop/ColumnChunkPageWriteStore.java       | 10 ++--
 .../parquet/hadoop/DirectCodecFactory.java      | 11 ++--
 .../hadoop/InternalParquetRecordReader.java     | 28 +++++-----
 .../hadoop/InternalParquetRecordWriter.java     | 16 +++---
 .../org/apache/parquet/hadoop/LruCache.java     | 34 ++++++------
 .../apache/parquet/hadoop/MemoryManager.java    |  7 +--
 .../parquet/hadoop/ParquetFileReader.java       | 36 +++++--------
 .../parquet/hadoop/ParquetFileWriter.java       | 41 +++++++--------
 .../parquet/hadoop/ParquetInputFormat.java      | 30 +++++------
 .../parquet/hadoop/ParquetOutputCommitter.java  |  5 +-
 .../parquet/hadoop/ParquetOutputFormat.java     | 28 +++++-----
 .../parquet/hadoop/ParquetRecordReader.java     |  5 +-
 .../hadoop/UnmaterializableRecordCounter.java   |  5 +-
 .../parquet/hadoop/codec/CodecConfig.java       | 18 +++----
 .../parquet/hadoop/util/HadoopStreams.java      |  6 ++-
 .../parquet/hadoop/util/SerializationUtil.java  |  5 +-
 .../parquet/hadoop/TestParquetFileWriter.java   |  7 +--
 .../hadoop/example/TestInputOutputFormat.java   |  9 ++--
 .../src/test/resources/log4j.properties         | 24 +++++++++
 .../parquet/hive/internal/Hive010Binding.java   |  5 +-
 .../parquet/hive/internal/Hive012Binding.java   |  5 +-
 .../apache/parquet/hive/HiveBindingFactory.java | 15 +++---
 .../io/parquet/MapredParquetOutputFormat.java   |  8 +--
 .../read/ParquetRecordReaderWrapper.java        |  6 +--
 .../write/ParquetRecordWriterWrapper.java       | 10 ++--
 .../org/apache/parquet/pig/ParquetLoader.java   | 32 ++++++------
 .../apache/parquet/pig/PigSchemaConverter.java  | 18 +++----
 .../apache/parquet/pig/TupleReadSupport.java    |  5 +-
 .../java/org/apache/parquet/pig/PerfTest2.java  |  8 ++-
 .../parquet/pig/TestTupleRecordConsumer.java    | 13 ++---
 .../parquet/pig/TupleConsumerPerfTest.java      |  1 -
 .../apache/parquet/proto/ProtoReadSupport.java  | 11 ++--
 .../parquet/proto/ProtoSchemaConverter.java     |  5 +-
 .../apache/parquet/proto/ProtoWriteSupport.java |  5 +-
 .../parquet/proto/utils/WriteUsingMR.java       |  9 ++--
 .../thrift/AbstractThriftWriteSupport.java      |  5 +-
 .../hadoop/thrift/ThriftReadSupport.java        |  9 ++--
 .../parquet/thrift/ParquetReadProtocol.java     | 47 +++++++++--------
 .../parquet/thrift/ParquetWriteProtocol.java    | 48 ++++++++---------
 .../apache/parquet/thrift/ThriftMetaData.java   |  5 +-
 .../parquet/thrift/ThriftRecordConverter.java   |  5 +-
 .../projection/StrictFieldProjectionFilter.java |  5 +-
 .../hadoop/thrift/TestInputOutputFormat.java    |  9 ++--
 ...stParquetToThriftReadWriteAndProjection.java |  7 +--
 .../thrift/TestThriftToParquetFileWriter.java   |  8 +--
 .../parquet/thrift/TestParquetReadProtocol.java |  7 +--
 .../thrift/TestParquetWriteProtocol.java        | 13 ++---
 .../parquet/tools/command/MergeCommand.java     |  2 -
 pom.xml                                         |  1 -
 100 files changed, 670 insertions(+), 646 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-avro/src/test/java/org/apache/parquet/avro/TestInputOutputFormat.java
----------------------------------------------------------------------
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 36c090f..7ba6c9b 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.Mapper;
 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 class TestInputOutputFormat {
   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());
     }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-avro/src/test/java/org/apache/parquet/avro/TestReflectInputOutputFormat.java
----------------------------------------------------------------------
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 3e1d32e..729f24a 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.io.Text;
 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.apache.parquet.filter.UnboundRecordFilter;
 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.assertTrue;
 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 class TestReflectInputOutputFormat {
   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());
     }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-avro/src/test/java/org/apache/parquet/avro/TestSpecificInputOutputFormat.java
----------------------------------------------------------------------
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 17a0af1..a0b58f3 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.apache.hadoop.mapreduce.lib.input.TextInputFormat;
 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 class TestSpecificInputOutputFormat {
   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());
     }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/CorruptDeltaByteArrays.java
----------------------------------------------------------------------
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 258c9ee..d19e489 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 @@ package org.apache.parquet;
 
 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 class CorruptDeltaByteArrays {
 
     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 class CorruptDeltaByteArrays {
 
     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 class CorruptDeltaByteArrays {
     }
 
     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;
     }
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/CorruptStatistics.java
----------------------------------------------------------------------
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 3b90338..3e3aa3c 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.SemanticVersion.SemanticVersionParseException;
 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 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
 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

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderImpl.java
----------------------------------------------------------------------
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 3fc327e..931b4b1 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.io.IOException;
 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.Binary;
 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 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeNameConverter;
  *
  */
 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 @@ public class ColumnReaderImpl implements ColumnReader {
   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 @@ public class ColumnReaderImpl implements ColumnReader {
   }
 
   private void readPage() {
-    if (DEBUG) LOG.debug("loading page");
+    LOG.debug("loading page");
     DataPage page = pageReader.readPage();
     page.accept(new DataPage.Visitor<Void>() {
       @Override
@@ -590,14 +590,14 @@ public class ColumnReaderImpl implements ColumnReader {
     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 @@ public class ColumnReaderImpl implements ColumnReader {
     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);

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV1.java
----------------------------------------------------------------------
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 dc6ebec..c5b3884 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 static org.apache.parquet.bytes.BytesInput.concat;
 
 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.statistics.Statistics;
 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 @@ import static java.lang.Math.max;
  *
  */
 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 @@ final class ColumnWriterV1 implements ColumnWriter {
   }
 
   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() {

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV2.java
----------------------------------------------------------------------
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 396d53a..c6fd91b 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.ValuesWriter;
 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 @@ import org.apache.parquet.io.api.Binary;
  *
  */
 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 @@ final class ColumnWriterV2 implements ColumnWriter {
   }
 
   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() {

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/BitPackingValuesReader.java
----------------------------------------------------------------------
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 f540c39..a5608cb 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.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.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 @@ import org.apache.parquet.io.ParquetDecodingException;
  *
  */
 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 class BitPackingValuesReader extends ValuesReader {
   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;

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPackingValuesReader.java
----------------------------------------------------------------------
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 f4c8c8e..7c19340 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.io.IOException;
 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 class ByteBitPackingValuesReader extends ValuesReader {
       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;

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesReader.java
----------------------------------------------------------------------
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 41f221d..d810ba8 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 @@ import org.apache.parquet.io.api.Binary;
  */
 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 class DeltaLengthByteArrayValuesReader 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));
     lengthReader.initFromPage(valueCount, in, offset);
     offset = lengthReader.getNextOffset();
     this.in = in;

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesWriter.java
----------------------------------------------------------------------
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 f7ad912..118153c 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 @@ package org.apache.parquet.column.values.deltalengthbytearray;
 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.DeltaBinaryPackingValuesWriter;
 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 @@ import org.apache.parquet.io.api.Binary;
  */
 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 class DeltaLengthByteArrayValuesWriter extends ValuesWriter {
     } 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));
   }
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesReader.java
----------------------------------------------------------------------
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 e421da9..19ff47c 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 @@ import org.apache.parquet.io.api.Binary;
  *
  */
 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 class DictionaryValuesReader extends ValuesReader {
       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) {

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesWriter.java
----------------------------------------------------------------------
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 86edd79..5ef7712 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.Iterator;
 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.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;
+
 /**
  * Will attempt to encode values using a dictionary and fall back to plain encoding
  *  if the dictionary gets too big
@@ -64,7 +65,7 @@ import org.apache.parquet.io.api.Binary;
  *
  */
 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 abstract class DictionaryValuesWriter extends ValuesWriter implements Req
   @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 abstract class DictionaryValuesWriter extends ValuesWriter implements Req
       // 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();

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BinaryPlainValuesReader.java
----------------------------------------------------------------------
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 26f5e29..82e5551 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 class BinaryPlainValuesReader 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 = in;
     this.offset = offset;
   }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BooleanPlainValuesReader.java
----------------------------------------------------------------------
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 a279938..1f8fc2c 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 @@ import org.apache.parquet.column.values.bitpacking.ByteBitPackingValuesReader;
  *
  */
 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 class BooleanPlainValuesReader 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.initFromPage(valueCount, in, offset);
   }
   

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesReader.java
----------------------------------------------------------------------
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 8496e7e..7a14f81 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 @@ package org.apache.parquet.column.values.plain;
 
 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 @@ import static org.apache.parquet.Log.DEBUG;
  * @author David Z. Chen <dchen@linkedin.com>
  */
 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 class FixedLenByteArrayPlainValuesReader 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 = in;
     this.offset = offset;
   }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesWriter.java
----------------------------------------------------------------------
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 6ab2dea..d7b2deb 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 @@ package org.apache.parquet.column.values.plain;
 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.values.ValuesWriter;
 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 @@ import org.apache.parquet.io.api.Binary;
  * @author David Z. Chen <dchen@linkedin.com>
  */
 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 class FixedLenByteArrayPlainValuesWriter extends ValuesWriter {
     } 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);
   }
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesReader.java
----------------------------------------------------------------------
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 c8fb303..e79cbb2 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 @@ import org.apache.parquet.io.ParquetDecodingException;
  *
  */
 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));
   }
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java
----------------------------------------------------------------------
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 add5495..aa96cb6 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.io.IOException;
 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.Encoding;
 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 @@ import org.apache.parquet.io.api.Binary;
  *
  */
 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 class PlainValuesWriter extends ValuesWriter {
     } 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);
   }
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridDecoder.java
----------------------------------------------------------------------
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 1280e8d..6daa349 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.io.InputStream;
 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 @@ import org.apache.parquet.io.ParquetDecodingException;
  * @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 @@ public class RunLengthBitPackingHybridDecoder {
   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 @@ public class RunLengthBitPackingHybridDecoder {
     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.

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridEncoder.java
----------------------------------------------------------------------
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 001d3f6..5fba70a 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 @@ package org.apache.parquet.column.values.rle;
 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 @@ import static org.apache.parquet.Log.DEBUG;
  * @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");
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/example/data/Group.java
----------------------------------------------------------------------
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 3fb7d4d..61f6317 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 @@ abstract public class Group extends GroupValueSource {
   }
 
   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));
   }
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/filter2/compat/FilterCompat.java
----------------------------------------------------------------------
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 2efcc39..17bd2e1 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 @@ import static org.apache.parquet.Preconditions.checkNotNull;
  * 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 class FilterCompat {
   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);

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/io/BaseRecordReader.java
----------------------------------------------------------------------
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 f2d88fc..8c7f390 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<T> extends RecordReader<T> {
-  private static final Log LOG = Log.getLog(BaseRecordReader.class);
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRecordReader.class);
 
   public RecordConsumer recordConsumer;
   public RecordMaterializer<T> recordMaterializer;
@@ -48,11 +48,11 @@ public abstract class BaseRecordReader<T> extends RecordReader<T> {
   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 @@ public abstract class BaseRecordReader<T> extends RecordReader<T> {
   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 @@ public abstract class BaseRecordReader<T> extends RecordReader<T> {
 
   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 @@ public abstract class BaseRecordReader<T> extends RecordReader<T> {
 
   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 @@ public abstract class BaseRecordReader<T> extends RecordReader<T> {
       recordConsumer.endField(endField, endIndex);
       endField = null;
     }
-    if (DEBUG) LOG.debug("endGroup()");
+    LOG.debug("endGroup()");
     recordConsumer.endGroup();
     endField(field, index);
   }
@@ -134,7 +134,7 @@ public abstract class BaseRecordReader<T> extends RecordReader<T> {
       recordConsumer.endField(endField, endIndex);
       endField = null;
     }
-    if (DEBUG) LOG.debug("endMessage()");
+    LOG.debug("endMessage()");
     recordConsumer.endMessage();
   }
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/io/ColumnIO.java
----------------------------------------------------------------------
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 95a969e..9c6e729 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 @@ package org.apache.parquet.io;
 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 @@ import org.apache.parquet.schema.Type.Repetition;
  */
 abstract public class ColumnIO {
 
-  static final boolean DEBUG = Log.DEBUG;
-
   private final GroupColumnIO parent;
   private final Type type;
   private final String name;

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/io/GroupColumnIO.java
----------------------------------------------------------------------
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 1efe0d1..14b8426 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.HashMap;
 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 @@ import org.apache.parquet.schema.GroupType;
  *
  */
 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<String, ColumnIO> childrenByName = new HashMap<String, ColumnIO>();
   private final List<ColumnIO> children = new ArrayList<ColumnIO>();

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/parquet-column/src/main/java/org/apache/parquet/io/MessageColumnIO.java
----------------------------------------------------------------------
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 f962105..67efdb3 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.HashMap;
 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 org.apache.parquet.schema.MessageType;
 
 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 @@ import static org.apache.parquet.Preconditions.checkNotNull;
  * @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<PrimitiveColumnIO> leaves;
 
@@ -261,20 +262,24 @@ public class MessageColumnIO extends GroupColumnIO {
       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 class MessageColumnIO extends GroupColumnIO {
     @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 class MessageColumnIO extends GroupColumnIO {
 
     @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 @@ public class MessageColumnIO extends GroupColumnIO {
           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 @@ public class MessageColumnIO extends GroupColumnIO {
 
     private void setRepetitionLevel() {
       r[currentLevel] = currentColumnIO.getRepetitionLevel();
-      if (DEBUG) log("r: " + r[currentLevel]);
+      if (DEBUG) log("r: {}", r[currentLevel]);
     }
 
     @Override
@@ -428,7 +432,7 @@ public class MessageColumnIO extends GroupColumnIO {
 
     @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 class MessageColumnIO extends GroupColumnIO {
 
     @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 class MessageColumnIO extends GroupColumnIO {
 
     @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 class MessageColumnIO extends GroupColumnIO {
 
     @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 class MessageColumnIO extends GroupColumnIO {
 
     @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 class MessageColumnIO extends GroupColumnIO {
 
     @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());
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/df9d8e41/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/df9d8e41/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/df9d8e41/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/df9d8e41/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 {
 


Mime
View raw message