aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala
blob: 0e36852ddd9b053fec65079e3d1b02b2444b8242 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.spark.sql.parquet

import java.io.IOException
import java.lang.{Long => JLong}
import java.text.SimpleDateFormat
import java.util.concurrent.{Callable, TimeUnit}
import java.util.{ArrayList, Collections, Date, List => JList}

import scala.collection.JavaConversions._
import scala.collection.mutable
import scala.util.Try

import com.google.common.cache.CacheBuilder
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{BlockLocation, FileStatus, Path}
import org.apache.hadoop.mapreduce._
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter, FileOutputFormat => NewFileOutputFormat}
import parquet.hadoop._
import parquet.hadoop.api.ReadSupport.ReadContext
import parquet.hadoop.api.{InitContext, ReadSupport}
import parquet.hadoop.metadata.GlobalMetaData
import parquet.hadoop.util.ContextUtil
import parquet.io.ParquetDecodingException
import parquet.schema.MessageType

import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.SQLConf
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row, _}
import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode}
import org.apache.spark.{Logging, SerializableWritable, TaskContext}

/**
 * :: DeveloperApi ::
 * Parquet table scan operator. Imports the file that backs the given
 * [[org.apache.spark.sql.parquet.ParquetRelation]] as a ``RDD[Row]``.
 */
case class ParquetTableScan(
    attributes: Seq[Attribute],
    relation: ParquetRelation,
    columnPruningPred: Seq[Expression])
  extends LeafNode {

  // The resolution of Parquet attributes is case sensitive, so we resolve the original attributes
  // by exprId. note: output cannot be transient, see
  // https://issues.apache.org/jira/browse/SPARK-1367
  val normalOutput =
    attributes
      .filterNot(a => relation.partitioningAttributes.map(_.exprId).contains(a.exprId))
      .flatMap(a => relation.output.find(o => o.exprId == a.exprId))

  val partOutput =
    attributes.flatMap(a => relation.partitioningAttributes.find(o => o.exprId == a.exprId))

  def output = partOutput ++ normalOutput

  assert(normalOutput.size + partOutput.size == attributes.size,
    s"$normalOutput + $partOutput != $attributes, ${relation.output}")

  override def execute(): RDD[Row] = {
    import parquet.filter2.compat.FilterCompat.FilterPredicateCompat

    val sc = sqlContext.sparkContext
    val job = new Job(sc.hadoopConfiguration)
    ParquetInputFormat.setReadSupportClass(job, classOf[RowReadSupport])

    val conf: Configuration = ContextUtil.getConfiguration(job)

    relation.path.split(",").foreach { curPath =>
      val qualifiedPath = {
        val path = new Path(curPath)
        path.getFileSystem(conf).makeQualified(path)
      }
      NewFileInputFormat.addInputPath(job, qualifiedPath)
    }

    // Store both requested and original schema in `Configuration`
    conf.set(
      RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
      ParquetTypesConverter.convertToString(normalOutput))
    conf.set(
      RowWriteSupport.SPARK_ROW_SCHEMA,
      ParquetTypesConverter.convertToString(relation.output))

    // Store record filtering predicate in `Configuration`
    // Note 1: the input format ignores all predicates that cannot be expressed
    // as simple column predicate filters in Parquet. Here we just record
    // the whole pruning predicate.
    ParquetFilters
      .createRecordFilter(columnPruningPred)
      .map(_.asInstanceOf[FilterPredicateCompat].getFilterPredicate)
      // Set this in configuration of ParquetInputFormat, needed for RowGroupFiltering
      .foreach(ParquetInputFormat.setFilterPredicate(conf, _))

    // Tell FilteringParquetRowInputFormat whether it's okay to cache Parquet and FS metadata
    conf.set(
      SQLConf.PARQUET_CACHE_METADATA,
      sqlContext.getConf(SQLConf.PARQUET_CACHE_METADATA, "true"))

    val baseRDD =
      new org.apache.spark.rdd.NewHadoopRDD(
        sc,
        classOf[FilteringParquetRowInputFormat],
        classOf[Void],
        classOf[Row],
        conf)

    if (partOutput.nonEmpty) {
      baseRDD.mapPartitionsWithInputSplit { case (split, iter) =>
        val partValue = "([^=]+)=([^=]+)".r
        val partValues =
          split.asInstanceOf[parquet.hadoop.ParquetInputSplit]
            .getPath
            .toString
            .split("/")
            .flatMap {
              case partValue(key, value) => Some(key -> value)
              case _ => None
            }.toMap

        val partitionRowValues =
          partOutput.map(a => Cast(Literal(partValues(a.name)), a.dataType).eval(EmptyRow))

        new Iterator[Row] {
          private[this] val joinedRow = new JoinedRow5(Row(partitionRowValues:_*), null)

          def hasNext = iter.hasNext

          def next() = joinedRow.withRight(iter.next()._2)
        }
      }
    } else {
      baseRDD.map(_._2)
    }
  }

  /**
   * Applies a (candidate) projection.
   *
   * @param prunedAttributes The list of attributes to be used in the projection.
   * @return Pruned TableScan.
   */
  def pruneColumns(prunedAttributes: Seq[Attribute]): ParquetTableScan = {
    val success = validateProjection(prunedAttributes)
    if (success) {
      ParquetTableScan(prunedAttributes, relation, columnPruningPred)
    } else {
      sys.error("Warning: Could not validate Parquet schema projection in pruneColumns")
    }
  }

  /**
   * Evaluates a candidate projection by checking whether the candidate is a subtype
   * of the original type.
   *
   * @param projection The candidate projection.
   * @return True if the projection is valid, false otherwise.
   */
  private def validateProjection(projection: Seq[Attribute]): Boolean = {
    val original: MessageType = relation.parquetSchema
    val candidate: MessageType = ParquetTypesConverter.convertFromAttributes(projection)
    Try(original.checkContains(candidate)).isSuccess
  }
}

/**
 * :: DeveloperApi ::
 * Operator that acts as a sink for queries on RDDs and can be used to
 * store the output inside a directory of Parquet files. This operator
 * is similar to Hive's INSERT INTO TABLE operation in the sense that
 * one can choose to either overwrite or append to a directory. Note
 * that consecutive insertions to the same table must have compatible
 * (source) schemas.
 *
 * WARNING: EXPERIMENTAL! InsertIntoParquetTable with overwrite=false may
 * cause data corruption in the case that multiple users try to append to
 * the same table simultaneously. Inserting into a table that was
 * previously generated by other means (e.g., by creating an HDFS
 * directory and importing Parquet files generated by other tools) may
 * cause unpredicted behaviour and therefore results in a RuntimeException
 * (only detected via filename pattern so will not catch all cases).
 */
@DeveloperApi
case class InsertIntoParquetTable(
    relation: ParquetRelation,
    child: SparkPlan,
    overwrite: Boolean = false)
  extends UnaryNode with SparkHadoopMapReduceUtil {

  /**
   * Inserts all rows into the Parquet file.
   */
  override def execute() = {
    // TODO: currently we do not check whether the "schema"s are compatible
    // That means if one first creates a table and then INSERTs data with
    // and incompatible schema the execution will fail. It would be nice
    // to catch this early one, maybe having the planner validate the schema
    // before calling execute().

    val childRdd = child.execute()
    assert(childRdd != null)

    val job = new Job(sqlContext.sparkContext.hadoopConfiguration)

    val writeSupport =
      if (child.output.map(_.dataType).forall(_.isPrimitive)) {
        log.debug("Initializing MutableRowWriteSupport")
        classOf[org.apache.spark.sql.parquet.MutableRowWriteSupport]
      } else {
        classOf[org.apache.spark.sql.parquet.RowWriteSupport]
      }

    ParquetOutputFormat.setWriteSupportClass(job, writeSupport)

    val conf = ContextUtil.getConfiguration(job)
    RowWriteSupport.setSchema(relation.output, conf)

    val fspath = new Path(relation.path)
    val fs = fspath.getFileSystem(conf)

    if (overwrite) {
      try {
        fs.delete(fspath, true)
      } catch {
        case e: IOException =>
          throw new IOException(
            s"Unable to clear output directory ${fspath.toString} prior"
              + s" to InsertIntoParquetTable:\n${e.toString}")
      }
    }
    saveAsHadoopFile(childRdd, relation.path.toString, conf)

    // We return the child RDD to allow chaining (alternatively, one could return nothing).
    childRdd
  }

  override def output = child.output

  /**
   * Stores the given Row RDD as a Hadoop file.
   *
   * Note: We cannot use ``saveAsNewAPIHadoopFile`` from [[org.apache.spark.rdd.PairRDDFunctions]]
   * together with [[org.apache.spark.util.MutablePair]] because ``PairRDDFunctions`` uses
   * ``Tuple2`` and not ``Product2``. Also, we want to allow appending files to an existing
   * directory and need to determine which was the largest written file index before starting to
   * write.
   *
   * @param rdd The [[org.apache.spark.rdd.RDD]] to writer
   * @param path The directory to write to.
   * @param conf A [[org.apache.hadoop.conf.Configuration]].
   */
  private def saveAsHadoopFile(
      rdd: RDD[Row],
      path: String,
      conf: Configuration) {
    val job = new Job(conf)
    val keyType = classOf[Void]
    job.setOutputKeyClass(keyType)
    job.setOutputValueClass(classOf[Row])
    NewFileOutputFormat.setOutputPath(job, new Path(path))
    val wrappedConf = new SerializableWritable(job.getConfiguration)
    val formatter = new SimpleDateFormat("yyyyMMddHHmm")
    val jobtrackerID = formatter.format(new Date())
    val stageId = sqlContext.sparkContext.newRddId()

    val taskIdOffset =
      if (overwrite) {
        1
      } else {
        FileSystemHelper
          .findMaxTaskId(NewFileOutputFormat.getOutputPath(job).toString, job.getConfiguration) + 1
      }

    def writeShard(context: TaskContext, iter: Iterator[Row]): Int = {
      // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it
      // around by taking a mod. We expect that no task will be attempted 2 billion times.
      val attemptNumber = (context.attemptId % Int.MaxValue).toInt
      /* "reduce task" <split #> <attempt # = spark task #> */
      val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId,
        attemptNumber)
      val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId)
      val format = new AppendingParquetOutputFormat(taskIdOffset)
      val committer = format.getOutputCommitter(hadoopContext)
      committer.setupTask(hadoopContext)
      val writer = format.getRecordWriter(hadoopContext)
      while (iter.hasNext) {
        val row = iter.next()
        writer.write(null, row)
      }
      writer.close(hadoopContext)
      committer.commitTask(hadoopContext)
      1
    }
    val jobFormat = new AppendingParquetOutputFormat(taskIdOffset)
    /* apparently we need a TaskAttemptID to construct an OutputCommitter;
     * however we're only going to use this local OutputCommitter for
     * setupJob/commitJob, so we just use a dummy "map" task.
     */
    val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = true, 0, 0)
    val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId)
    val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext)
    jobCommitter.setupJob(jobTaskContext)
    sqlContext.sparkContext.runJob(rdd, writeShard _)
    jobCommitter.commitJob(jobTaskContext)
  }
}

/**
 * TODO: this will be able to append to directories it created itself, not necessarily
 * to imported ones.
 */
private[parquet] class AppendingParquetOutputFormat(offset: Int)
  extends parquet.hadoop.ParquetOutputFormat[Row] {
  // override to accept existing directories as valid output directory
  override def checkOutputSpecs(job: JobContext): Unit = {}

  // override to choose output filename so not overwrite existing ones
  override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = {
    val taskId: TaskID = getTaskAttemptID(context).getTaskID
    val partition: Int = taskId.getId
    val filename = s"part-r-${partition + offset}.parquet"
    val committer: FileOutputCommitter =
      getOutputCommitter(context).asInstanceOf[FileOutputCommitter]
    new Path(committer.getWorkPath, filename)
  }

  // The TaskAttemptContext is a class in hadoop-1 but is an interface in hadoop-2.
  // The signatures of the method TaskAttemptContext.getTaskAttemptID for the both versions
  // are the same, so the method calls are source-compatible but NOT binary-compatible because
  // the opcode of method call for class is INVOKEVIRTUAL and for interface is INVOKEINTERFACE.
  private def getTaskAttemptID(context: TaskAttemptContext): TaskAttemptID = {
    context.getClass.getMethod("getTaskAttemptID").invoke(context).asInstanceOf[TaskAttemptID]
  }
}

/**
 * We extend ParquetInputFormat in order to have more control over which
 * RecordFilter we want to use.
 */
private[parquet] class FilteringParquetRowInputFormat
  extends parquet.hadoop.ParquetInputFormat[Row] with Logging {

  private var footers: JList[Footer] = _

  private var fileStatuses = Map.empty[Path, FileStatus]

  override def createRecordReader(
      inputSplit: InputSplit,
      taskAttemptContext: TaskAttemptContext): RecordReader[Void, Row] = {

    import parquet.filter2.compat.FilterCompat.NoOpFilter

    val readSupport: ReadSupport[Row] = new RowReadSupport()

    val filter = ParquetInputFormat.getFilter(ContextUtil.getConfiguration(taskAttemptContext))
    if (!filter.isInstanceOf[NoOpFilter]) {
      new ParquetRecordReader[Row](
        readSupport,
        filter)
    } else {
      new ParquetRecordReader[Row](readSupport)
    }
  }

  override def getFooters(jobContext: JobContext): JList[Footer] = {
    import org.apache.spark.sql.parquet.FilteringParquetRowInputFormat.footerCache

    if (footers eq null) {
      val conf = ContextUtil.getConfiguration(jobContext)
      val cacheMetadata = conf.getBoolean(SQLConf.PARQUET_CACHE_METADATA, true)
      val statuses = listStatus(jobContext)
      fileStatuses = statuses.map(file => file.getPath -> file).toMap
      if (statuses.isEmpty) {
        footers = Collections.emptyList[Footer]
      } else if (!cacheMetadata) {
        // Read the footers from HDFS
        footers = getFooters(conf, statuses)
      } else {
        // Read only the footers that are not in the footerCache
        val foundFooters = footerCache.getAllPresent(statuses)
        val toFetch = new ArrayList[FileStatus]
        for (s <- statuses) {
          if (!foundFooters.containsKey(s)) {
            toFetch.add(s)
          }
        }
        val newFooters = new mutable.HashMap[FileStatus, Footer]
        if (toFetch.size > 0) {
          val startFetch = System.currentTimeMillis
          val fetched = getFooters(conf, toFetch)
          logInfo(s"Fetched $toFetch footers in ${System.currentTimeMillis - startFetch} ms")
          for ((status, i) <- toFetch.zipWithIndex) {
            newFooters(status) = fetched.get(i)
          }
          footerCache.putAll(newFooters)
        }
        footers = new ArrayList[Footer](statuses.size)
        for (status <- statuses) {
          footers.add(newFooters.getOrElse(status, foundFooters.get(status)))
        }
      }
    }

    footers
  }

  // TODO Remove this method and related code once PARQUET-16 is fixed
  // This method together with the `getFooters` method and the `fileStatuses` field are just used
  // to mimic this PR: https://github.com/apache/incubator-parquet-mr/pull/17
  override def getSplits(
      configuration: Configuration,
      footers: JList[Footer]): JList[ParquetInputSplit] = {

    // Use task side strategy by default
    val taskSideMetaData = configuration.getBoolean(ParquetInputFormat.TASK_SIDE_METADATA, true)
    val maxSplitSize: JLong = configuration.getLong("mapred.max.split.size", Long.MaxValue)
    val minSplitSize: JLong =
      Math.max(getFormatMinSplitSize, configuration.getLong("mapred.min.split.size", 0L))
    if (maxSplitSize < 0 || minSplitSize < 0) {
      throw new ParquetDecodingException(
        s"maxSplitSize or minSplitSie should not be negative: maxSplitSize = $maxSplitSize;" +
          s" minSplitSize = $minSplitSize")
    }

    // Uses strict type checking by default
    val getGlobalMetaData =
      classOf[ParquetFileWriter].getDeclaredMethod("getGlobalMetaData", classOf[JList[Footer]])
    getGlobalMetaData.setAccessible(true)
    val globalMetaData = getGlobalMetaData.invoke(null, footers).asInstanceOf[GlobalMetaData]

    if (globalMetaData == null) {
     val splits = mutable.ArrayBuffer.empty[ParquetInputSplit]
     return splits
    }

    val readContext = getReadSupport(configuration).init(
      new InitContext(configuration,
        globalMetaData.getKeyValueMetaData,
        globalMetaData.getSchema))

    if (taskSideMetaData){
      logInfo("Using Task Side Metadata Split Strategy")
      getTaskSideSplits(configuration,
        footers,
        maxSplitSize,
        minSplitSize,
        readContext)
    } else {
      logInfo("Using Client Side Metadata Split Strategy")
      getClientSideSplits(configuration,
        footers,
        maxSplitSize,
        minSplitSize,
        readContext)
    }

  }

  def getClientSideSplits(
    configuration: Configuration,
    footers: JList[Footer],
    maxSplitSize: JLong,
    minSplitSize: JLong,
    readContext: ReadContext): JList[ParquetInputSplit] = {

    import parquet.filter2.compat.FilterCompat.Filter
    import parquet.filter2.compat.RowGroupFilter
    import org.apache.spark.sql.parquet.FilteringParquetRowInputFormat.blockLocationCache

    val cacheMetadata = configuration.getBoolean(SQLConf.PARQUET_CACHE_METADATA, true)

    val splits = mutable.ArrayBuffer.empty[ParquetInputSplit]
    val filter: Filter = ParquetInputFormat.getFilter(configuration)
    var rowGroupsDropped: Long = 0
    var totalRowGroups: Long  = 0

    // Ugly hack, stuck with it until PR:
    // https://github.com/apache/incubator-parquet-mr/pull/17
    // is resolved
    val generateSplits =
      Class.forName("parquet.hadoop.ClientSideMetadataSplitStrategy")
       .getDeclaredMethods.find(_.getName == "generateSplits").getOrElse(
         sys.error(s"Failed to reflectively invoke ClientSideMetadataSplitStrategy.generateSplits"))
    generateSplits.setAccessible(true)

    for (footer <- footers) {
      val fs = footer.getFile.getFileSystem(configuration)
      val file = footer.getFile
      val status = fileStatuses.getOrElse(file, fs.getFileStatus(file))
      val parquetMetaData = footer.getParquetMetadata
      val blocks = parquetMetaData.getBlocks
      totalRowGroups = totalRowGroups + blocks.size
      val filteredBlocks = RowGroupFilter.filterRowGroups(
        filter,
        blocks,
        parquetMetaData.getFileMetaData.getSchema)
      rowGroupsDropped = rowGroupsDropped + (blocks.size - filteredBlocks.size)

      if (!filteredBlocks.isEmpty){
          var blockLocations: Array[BlockLocation] = null
          if (!cacheMetadata) {
            blockLocations = fs.getFileBlockLocations(status, 0, status.getLen)
          } else {
            blockLocations = blockLocationCache.get(status, new Callable[Array[BlockLocation]] {
              def call(): Array[BlockLocation] = fs.getFileBlockLocations(status, 0, status.getLen)
            })
          }
          splits.addAll(
            generateSplits.invoke(
              null,
              filteredBlocks,
              blockLocations,
              status,
              readContext.getRequestedSchema.toString,
              readContext.getReadSupportMetadata,
              minSplitSize,
              maxSplitSize).asInstanceOf[JList[ParquetInputSplit]])
        }
    }

    if (rowGroupsDropped > 0 && totalRowGroups > 0){
      val percentDropped = ((rowGroupsDropped/totalRowGroups.toDouble) * 100).toInt
      logInfo(s"Dropping $rowGroupsDropped row groups that do not pass filter predicate "
        + s"($percentDropped %) !")
    }
    else {
      logInfo("There were no row groups that could be dropped due to filter predicates")
    }
    splits

  }

  def getTaskSideSplits(
    configuration: Configuration,
    footers: JList[Footer],
    maxSplitSize: JLong,
    minSplitSize: JLong,
    readContext: ReadContext): JList[ParquetInputSplit] = {

    val splits = mutable.ArrayBuffer.empty[ParquetInputSplit]

    // Ugly hack, stuck with it until PR:
    // https://github.com/apache/incubator-parquet-mr/pull/17
    // is resolved
    val generateSplits =
      Class.forName("parquet.hadoop.TaskSideMetadataSplitStrategy")
       .getDeclaredMethods.find(_.getName == "generateTaskSideMDSplits").getOrElse(
         sys.error(
           s"Failed to reflectively invoke TaskSideMetadataSplitStrategy.generateTaskSideMDSplits"))
    generateSplits.setAccessible(true)

    for (footer <- footers) {
      val file = footer.getFile
      val fs = file.getFileSystem(configuration)
      val status = fileStatuses.getOrElse(file, fs.getFileStatus(file))
      val blockLocations = fs.getFileBlockLocations(status, 0, status.getLen)
      splits.addAll(
        generateSplits.invoke(
         null,
         blockLocations,
         status,
         readContext.getRequestedSchema.toString,
         readContext.getReadSupportMetadata,
         minSplitSize,
         maxSplitSize).asInstanceOf[JList[ParquetInputSplit]])
    }

    splits
  }

}

private[parquet] object FilteringParquetRowInputFormat {
  private val footerCache = CacheBuilder.newBuilder()
    .maximumSize(20000)
    .build[FileStatus, Footer]()

  private val blockLocationCache = CacheBuilder.newBuilder()
    .maximumSize(20000)
    .expireAfterWrite(15, TimeUnit.MINUTES)  // Expire locations since HDFS files might move
    .build[FileStatus, Array[BlockLocation]]()
}

private[parquet] object FileSystemHelper {
  def listFiles(pathStr: String, conf: Configuration): Seq[Path] = {
    val origPath = new Path(pathStr)
    val fs = origPath.getFileSystem(conf)
    if (fs == null) {
      throw new IllegalArgumentException(
        s"ParquetTableOperations: Path $origPath is incorrectly formatted")
    }
    val path = origPath.makeQualified(fs)
    if (!fs.exists(path) || !fs.getFileStatus(path).isDir) {
      throw new IllegalArgumentException(
        s"ParquetTableOperations: path $path does not exist or is not a directory")
    }
    fs.listStatus(path).map(_.getPath)
  }

    /**
     * Finds the maximum taskid in the output file names at the given path.
     */
  def findMaxTaskId(pathStr: String, conf: Configuration): Int = {
    val files = FileSystemHelper.listFiles(pathStr, conf)
    // filename pattern is part-r-<int>.parquet
    val nameP = new scala.util.matching.Regex("""part-r-(\d{1,}).parquet""", "taskid")
    val hiddenFileP = new scala.util.matching.Regex("_.*")
    files.map(_.getName).map {
      case nameP(taskid) => taskid.toInt
      case hiddenFileP() => 0
      case other: String =>
        sys.error("ERROR: attempting to append to set of Parquet files and found file" +
          s"that does not match name pattern: $other")
      case _ => 0
    }.reduceLeft((a, b) => if (a < b) b else a)
  }
}