private CodecPool()

in parquet-hadoop/src/main/java/org/apache/parquet/hadoop/DirectCodecFactory.java [487:563]


      private CodecPool(final CompressionCodec codec) {
        try {
          boolean supportDirectDecompressor = DIRECT_DECOMPRESSION_CODEC_CLASS != null
              && DIRECT_DECOMPRESSION_CODEC_CLASS.isAssignableFrom(codec.getClass());
          compressorPool = new GenericObjectPool(
              new BasePoolableObjectFactory() {
                public Object makeObject() throws Exception {
                  return codec.createCompressor();
                }
              },
              Integer.MAX_VALUE);

          Object com = compressorPool.borrowObject();
          if (com != null) {
            cPools.put(com.getClass(), compressorPool);
            compressorPool.returnObject(com);
          } else {
            if (LOG.isDebugEnabled()) {
              LOG.debug(String.format(
                  BYTE_BUF_IMPL_NOT_FOUND_MSG,
                  "compressor",
                  codec.getClass().getName()));
            }
          }

          decompressorPool = new GenericObjectPool(
              new BasePoolableObjectFactory() {
                public Object makeObject() throws Exception {
                  return codec.createDecompressor();
                }
              },
              Integer.MAX_VALUE);

          Object decom = decompressorPool.borrowObject();
          if (decom != null) {
            dePools.put(decom.getClass(), decompressorPool);
            decompressorPool.returnObject(decom);
          } else {
            if (LOG.isDebugEnabled()) {
              LOG.debug(String.format(
                  BYTE_BUF_IMPL_NOT_FOUND_MSG,
                  "decompressor",
                  codec.getClass().getName()));
            }
          }

          if (supportDirectDecompressor) {
            directDecompressorPool = new GenericObjectPool(
                new BasePoolableObjectFactory() {
                  public Object makeObject() throws Exception {
                    return CREATE_DIRECT_DECOMPRESSOR_METHOD.invoke(codec);
                  }
                },
                Integer.MAX_VALUE);

            Object ddecom = directDecompressorPool.borrowObject();
            if (ddecom != null) {
              directDePools.put(ddecom.getClass(), directDecompressorPool);
              directDecompressorPool.returnObject(ddecom);
            } else {
              supportDirectDecompressor = false;
              if (LOG.isDebugEnabled()) {
                LOG.debug(String.format(
                    BYTE_BUF_IMPL_NOT_FOUND_MSG,
                    "compressor",
                    codec.getClass().getName()));
              }
            }
          } else {
            directDecompressorPool = null;
          }

          this.supportDirectDecompressor = supportDirectDecompressor;
        } catch (Exception e) {
          throw new ParquetCompressionCodecException("Error creating compression codec pool.", e);
        }
      }