diff --git a/lui-core/src/main/scala/com/twitter/lui/ColumnIO.scala b/lui-core/src/main/scala/com/twitter/lui/ColumnIO.scala new file mode 100644 index 0000000000..f5ba2cd44f --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/ColumnIO.scala @@ -0,0 +1,72 @@ +/* + * 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 com.twitter.lui + +import java.util.Arrays + +import org.apache.parquet.Log +import org.apache.parquet.schema.Type +import org.apache.parquet.schema.Type.Repetition + +abstract class ColumnIO { + def columnType: Type + def parent: GroupColumnIO + def index: Int + + def fieldPath: Array[String] + + def getFieldPath(level: Int): String = fieldPath(level) + + def indexFieldPath: Array[Int] + + def getIndexFieldPath(level: Int): Int = indexFieldPath(level) + + def name: String = columnType.getName + + def repetitionLevel: Int + + def definitionLevel: Int + + // def setLevels(r: Int, d: Int, fieldPath: Array[String], indexFieldPath: Array[Int], repetition: List[ColumnIO], path: List[ColumnIO]) { + // setRepetitionLevel(r) + // setDefinitionLevel(d) + // this.fieldPath = fieldPath + // this.indexFieldPath = indexFieldPath + // } + + def getColumnNames: Seq[Seq[String]] + + def last: PrimitiveColumnIO + def first: PrimitiveColumnIO + + def leaves: Iterator[PrimitiveColumnIO] + + def getParent(r: Int): ColumnIO = + if (repetitionLevel == r && columnType.isRepetition(Repetition.REPEATED)) { + this + } else if (parent != null && parent.definitionLevel >= r) { + parent.getParent(r) + } else { + throw new org.apache.parquet.io.InvalidRecordException(s"no parent($r) for $fieldPath") + } + + override def toString: String = + s"${getClass.getSimpleName} $name r: $repetitionLevel d: $definitionLevel $fieldPath" + +} diff --git a/lui-core/src/main/scala/com/twitter/lui/ColumnIOBuilder.scala b/lui-core/src/main/scala/com/twitter/lui/ColumnIOBuilder.scala new file mode 100644 index 0000000000..fe1487e8ed --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/ColumnIOBuilder.scala @@ -0,0 +1,39 @@ +/* + * 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 com.twitter.lui + +import java.util.Arrays + +import org.apache.parquet.Log +import org.apache.parquet.schema.Type +import org.apache.parquet.schema.Type.Repetition + +abstract class ColumnIOBuilder { + + def columnType: Type + def index: Int + + def setLevels(parentBox: LazyBox[GroupColumnIO], + r: Int, + d: Int, + fieldPath: Array[String], + indexFieldPath: Array[Int], + repetition: List[ColumnIOBuilder]): ColumnIO + +} diff --git a/lui-core/src/main/scala/com/twitter/lui/GroupColumnIO.scala b/lui-core/src/main/scala/com/twitter/lui/GroupColumnIO.scala new file mode 100644 index 0000000000..2666a0cb91 --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/GroupColumnIO.scala @@ -0,0 +1,67 @@ +/* + * 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 com.twitter.lui + +import org.apache.parquet.Log +import org.apache.parquet.schema.GroupType + +/** + * Group level of the IO structure + * + * + */ + +object GroupColumnIO { + private val logger: Log = Log.getLog(getClass) + private val DEBUG: Boolean = Log.DEBUG +} + +case class GroupColumnIO(columnType: GroupType, + parentBox: LazyBox[GroupColumnIO], + index: Int, + repetitionLevel: Int, + definitionLevel: Int, + fieldPath: Array[String], + indexFieldPath: Array[Int], + childrenByName: Map[String, ColumnIO], + children: Vector[ColumnIO]) extends ColumnIO { + + lazy val parent = parentBox.get + + import GroupColumnIO._ + + override def getColumnNames: Seq[Seq[String]] = children.flatMap(_.getColumnNames) + + override def last: PrimitiveColumnIO = children.last.last + + override def first: PrimitiveColumnIO = children.head.first + + def getChild(name: String): Option[ColumnIO] = childrenByName.get(name) + + def getChild(fieldIndex: Int): ColumnIO = + try { + children(fieldIndex) + } catch { + case e: IndexOutOfBoundsException => + throw new org.apache.parquet.io.InvalidRecordException(s"could not get child $fieldIndex from $children", e) + } + + override def leaves: Iterator[PrimitiveColumnIO] = children.toIterator.flatMap(_.leaves) + +} diff --git a/lui-core/src/main/scala/com/twitter/lui/GroupColumnIOBuilder.scala b/lui-core/src/main/scala/com/twitter/lui/GroupColumnIOBuilder.scala new file mode 100644 index 0000000000..20d85b61af --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/GroupColumnIOBuilder.scala @@ -0,0 +1,103 @@ +/* + * 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 com.twitter.lui + +import org.apache.parquet.schema.Type.Repetition.REPEATED +import org.apache.parquet.schema.Type.Repetition.REQUIRED + +import java.util.Arrays +import scala.collection.mutable.{ Map => MMap } + +import org.apache.parquet.Log +import org.apache.parquet.schema.GroupType + +object LazyBox { + def apply[T] = new LazyBox[T] +} +class LazyBox[T]() { + private[this] var contents: Option[T] = None + def get = this.synchronized { + if (contents.isEmpty) sys.error("Accessed Box contents before populated") + contents.get + } + def setValue(v: T): Unit = this.synchronized { + if (contents.isDefined) sys.error("Attempted to set box contents twice!") + contents = Some(v) + } +} + +case class GroupColumnIOBuilder( + columnType: GroupType, index: Int) extends ColumnIOBuilder { + private[this] var children = Vector[ColumnIOBuilder]() + + def add(child: ColumnIOBuilder) { + children = children :+ child + } + + override def setLevels(parentBox: LazyBox[GroupColumnIO], + repetitionLevel: Int, + definitionLevel: Int, + fieldPath: Array[String], + indexFieldPath: Array[Int], + repetition: List[ColumnIOBuilder]): GroupColumnIO = { + + val ourBox = LazyBox[GroupColumnIO] + + val newChildren: Vector[ColumnIO] = children.map { child => + val newFieldPath: Array[String] = Arrays.copyOf(fieldPath, fieldPath.length + 1) + val newIndexFieldPath: Array[Int] = Arrays.copyOf(indexFieldPath, indexFieldPath.length + 1) + newFieldPath(fieldPath.length) = child.columnType.getName + newIndexFieldPath(indexFieldPath.length) = child.index + + val newRepetition = if (child.columnType.isRepetition(REPEATED)) { + repetition :+ child + } else { + repetition + } + + child.setLevels( + ourBox, + // the type repetition level increases whenever there's a possible repetition + if (child.columnType.isRepetition(REPEATED)) repetitionLevel + 1 else repetitionLevel, + // the type definition level increases whenever a field can be missing (not required) + if (!child.columnType.isRepetition(REQUIRED)) definitionLevel + 1 else definitionLevel, + newFieldPath, + newIndexFieldPath, + newRepetition) + } + + val childrenByName: Map[String, ColumnIO] = newChildren.map { child => + child.columnType.getName -> child + }.toMap + + val gcIO = GroupColumnIO( + columnType, + parentBox, + index, + repetitionLevel, + definitionLevel, + fieldPath, + indexFieldPath, + childrenByName, + newChildren) + + ourBox.setValue(gcIO) + gcIO + } +} diff --git a/lui-core/src/main/scala/com/twitter/lui/MessageColumnIO.scala b/lui-core/src/main/scala/com/twitter/lui/MessageColumnIO.scala new file mode 100644 index 0000000000..e12f2f63db --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/MessageColumnIO.scala @@ -0,0 +1,45 @@ +/* + * 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 com.twitter.lui + +import java.util.Arrays + +import org.apache.parquet.Log +import org.apache.parquet.column.ColumnWriteStore +import org.apache.parquet.column.ColumnWriter +import org.apache.parquet.column.page.PageReadStore +import org.apache.parquet.column.values.dictionary.IntList +import org.apache.parquet.io.api.RecordMaterializer +import org.apache.parquet.schema.MessageType +import scala.collection.JavaConverters._ +import com.twitter.lui.hadoop.{ ReadSupport, ReadContext } +import org.apache.hadoop.conf.Configuration + +/** + * Message level of the IO structure + */ +object MessageColumnIO { + private val logger: Log = Log.getLog(getClass) + private val DEBUG: Boolean = Log.DEBUG +} + +case class MessageColumnIO(messageType: MessageType, + validating: Boolean, + createdBy: String, + root: GroupColumnIO) \ No newline at end of file diff --git a/lui-core/src/main/scala/com/twitter/lui/MessageColumnIOBuilder.scala b/lui-core/src/main/scala/com/twitter/lui/MessageColumnIOBuilder.scala new file mode 100644 index 0000000000..9e12de8c8e --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/MessageColumnIOBuilder.scala @@ -0,0 +1,41 @@ +/* + * 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 com.twitter.lui + +import java.util.Arrays + +import org.apache.parquet.Log +import org.apache.parquet.schema.MessageType +import scala.collection.JavaConverters._ + +case class MessageColumnIOBuilder(columnType: MessageType, validating: Boolean, createdBy: String) { + val rootColumnIOBuilder = GroupColumnIOBuilder(columnType, 0) + + def setLevels: MessageColumnIO = { + val ourBox = LazyBox[GroupColumnIO] + val definitionLevel = 0 + val repetitionLevel = 0 + val fieldPath = Array[String]() + val indexFieldPath = Array[Int]() + val repetition = List(rootColumnIOBuilder) + val r = rootColumnIOBuilder.setLevels(ourBox, definitionLevel, repetitionLevel, fieldPath, indexFieldPath, repetition) + ourBox.setValue(null) + MessageColumnIO(columnType, validating, createdBy, r) + } +} \ No newline at end of file diff --git a/lui-core/src/main/scala/com/twitter/lui/MessageColumnIOFactory.scala b/lui-core/src/main/scala/com/twitter/lui/MessageColumnIOFactory.scala new file mode 100644 index 0000000000..7beb2d308f --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/MessageColumnIOFactory.scala @@ -0,0 +1,119 @@ +/* + * 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 com.twitter.lui + +import com.twitter.lui.hadoop.{ ReadSupport, ReadContext } +import java.util.ArrayList +import org.apache.hadoop.conf.Configuration +import org.apache.parquet.column.page.PageReadStore +import org.apache.parquet.io.ParquetDecodingException +import org.apache.parquet.schema.GroupType +import org.apache.parquet.schema.MessageType +import org.apache.parquet.schema.PrimitiveType +import org.apache.parquet.schema.Type +import org.apache.parquet.schema.TypeVisitor +import scala.collection.JavaConverters._ + +/** + * Factory constructing the ColumnIO structure from the schema + * + */ + +object MessageColumnIOFactory { + + def getRecordReader[T]( + columns: PageReadStore, + configuration: Configuration, + createdBy: String, + fileMetadata: Map[String, String], + fileSchema: MessageType, + readContext: ReadContext, + readSupport: ReadSupport[T], + strictTypeChecking: Boolean): RecordReader[T] = + readSupport.getRecordReader(columns, configuration, createdBy, fileMetadata, fileSchema, readContext, strictTypeChecking) + + case class StrictTypeChecking(on: Boolean) + case class LeafIndex(var toInt: Int) { + def getAndIncr: Int = { + val old = toInt + toInt = old + 1 + old + } + } + + def buildMessageColumnIO( + readContext: ReadContext, + fileSchema: MessageType, + createdBy: String, + strictTypeChecking: Boolean): MessageColumnIO = { + val messageBuilder = MessageColumnIOBuilder(readContext.requestedSchema, false, createdBy) + + implicit val stC = StrictTypeChecking(strictTypeChecking) + implicit val leafIndex = LeafIndex(0) + + visitChildren(messageBuilder.rootColumnIOBuilder, fileSchema, readContext.requestedSchema.asGroupType) + + messageBuilder.setLevels + } + + private[this] def visitNode(parentBuilder: GroupColumnIOBuilder, + tpe: Type, + currentRequestedType: Type, + currentRequestedIndex: Int)(implicit strictTypeChecking: StrictTypeChecking, leafIndex: LeafIndex): Unit = { + tpe match { + case groupType: GroupType => + if (currentRequestedType.isPrimitive) { + incompatibleSchema(groupType, currentRequestedType) + } + val newGrpIOBuilder = GroupColumnIOBuilder(groupType, currentRequestedIndex) + parentBuilder.add(newGrpIOBuilder) + visitChildren(newGrpIOBuilder, groupType, currentRequestedType.asGroupType) + + case primitiveType: PrimitiveType => + if (!currentRequestedType.isPrimitive || + (strictTypeChecking.on && + currentRequestedType.asPrimitiveType.getPrimitiveTypeName != primitiveType.getPrimitiveTypeName)) { + incompatibleSchema(primitiveType, currentRequestedType) + } + val newIO = PrimitiveColumnIOBuilder(primitiveType, currentRequestedIndex, leafIndex.getAndIncr) + parentBuilder.add(newIO) + } + + } + + private[this] def visitChildren(groupIOBuilder: GroupColumnIOBuilder, groupType: GroupType, requestedGroupType: GroupType)(implicit strictTypeChecking: StrictTypeChecking, leafIndex: LeafIndex): Unit = + groupType.getFields.asScala.foreach { curTpe => + // if the file schema does not contain the field it will just stay null + if (requestedGroupType.containsField(curTpe.getName)) { + + val currentRequestedIndex: Int = requestedGroupType.getFieldIndex(curTpe.getName) + val currentRequestedType: Type = requestedGroupType.getType(currentRequestedIndex) + + if (currentRequestedType.getRepetition.isMoreRestrictiveThan(curTpe.getRepetition)) { + incompatibleSchema(curTpe, currentRequestedType) + } + visitNode(groupIOBuilder, curTpe, currentRequestedType, currentRequestedIndex) + } + } + + private[this] def incompatibleSchema(fileType: Type, requestedType: Type): Unit = { + throw new ParquetDecodingException("The requested schema is not compatible with the file schema. incompatible types: " + requestedType + " != " + fileType) + } + +} diff --git a/lui-core/src/main/scala/com/twitter/lui/PrimitiveColumnIO.scala b/lui-core/src/main/scala/com/twitter/lui/PrimitiveColumnIO.scala new file mode 100644 index 0000000000..57b1f5ade2 --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/PrimitiveColumnIO.scala @@ -0,0 +1,67 @@ +/* + * 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 com.twitter.lui + +import java.util.Arrays + +import org.apache.parquet.column.ColumnDescriptor +import org.apache.parquet.schema.Type +import org.apache.parquet.schema.PrimitiveType +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName + +/** + * Primitive level of the IO structure + * + */ +case class PrimitiveColumnIO(columnType: Type, + parentBox: LazyBox[GroupColumnIO], + id: Int, + index: Int, + repetitionLevel: Int, + definitionLevel: Int, + fieldPath: Array[String], + indexFieldPath: Array[Int], + columnDescriptor: ColumnDescriptor) extends ColumnIO { + + lazy val parent = parentBox.get + + lazy val path: Array[ColumnIO] = { + @annotation.tailrec + def go(p: ColumnIO, acc: List[ColumnIO]): List[ColumnIO] = + if (p.parent != null) go(p.parent, p :: acc) else p :: acc + + go(this, Nil).toArray + } + + override def getColumnNames: Seq[Seq[String]] = List(fieldPath) + + def getColumnDescriptor: ColumnDescriptor = columnDescriptor + + def isLast(r: Int): Boolean = getParent(r).last eq this + + def isFirst(r: Int): Boolean = getParent(r).first eq this + + override def last: PrimitiveColumnIO = this + + override def first: PrimitiveColumnIO = this + + def getPrimitive: PrimitiveTypeName = columnType.asPrimitiveType.getPrimitiveTypeName + + override def leaves: Iterator[PrimitiveColumnIO] = Iterator(this) +} diff --git a/lui-core/src/main/scala/com/twitter/lui/PrimitiveColumnIOBuilder.scala b/lui-core/src/main/scala/com/twitter/lui/PrimitiveColumnIOBuilder.scala new file mode 100644 index 0000000000..3cb342ad5c --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/PrimitiveColumnIOBuilder.scala @@ -0,0 +1,54 @@ +/* + * 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 com.twitter.lui + +import java.util.Arrays + +import org.apache.parquet.column.ColumnDescriptor +import org.apache.parquet.schema.Type +import org.apache.parquet.schema.PrimitiveType +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName + +case class PrimitiveColumnIOBuilder(columnType: Type, index: Int, val id: Int) extends ColumnIOBuilder { + override def setLevels(parentBox: LazyBox[GroupColumnIO], + repetitionLevel: Int, + definitionLevel: Int, + fieldPath: Array[String], + indexFieldPath: Array[Int], + repetition: List[ColumnIOBuilder]): PrimitiveColumnIO = { + val pType: PrimitiveType = columnType.asPrimitiveType + val columnDescriptor = new ColumnDescriptor( + fieldPath, + pType.getPrimitiveTypeName, + pType.getTypeLength, + repetitionLevel, + definitionLevel) + + PrimitiveColumnIO( + columnType, + parentBox, + id, + index, + repetitionLevel, + definitionLevel, + fieldPath, + indexFieldPath, + columnDescriptor) + } +} diff --git a/lui-core/src/main/scala/com/twitter/lui/RecordReader.scala b/lui-core/src/main/scala/com/twitter/lui/RecordReader.scala new file mode 100644 index 0000000000..859de0df64 --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/RecordReader.scala @@ -0,0 +1,32 @@ +/* + * 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 com.twitter.lui + +/** + * used to read reassembled records + * @param the type of the materialized record + */ +abstract class RecordReader[T] { + + /** + * Reads one record and returns it. + * @return the materialized record + */ + def read(): T +} diff --git a/lui-core/src/main/scala/com/twitter/lui/column_reader/BaseColumnReader.scala b/lui-core/src/main/scala/com/twitter/lui/column_reader/BaseColumnReader.scala new file mode 100644 index 0000000000..001c307b55 --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/column_reader/BaseColumnReader.scala @@ -0,0 +1,273 @@ +/* + * 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 com.twitter.lui.column_reader + +import java.lang.String.format +import org.apache.parquet.Log.DEBUG +import org.apache.parquet.Preconditions.checkNotNull +import org.apache.parquet.column.ValuesType.DEFINITION_LEVEL +import org.apache.parquet.column.ValuesType.REPETITION_LEVEL +import org.apache.parquet.column.ValuesType.VALUES + +import java.io.ByteArrayInputStream +import java.io.IOException + +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 +import org.apache.parquet.column.ColumnDescriptor +import org.apache.parquet.column.Dictionary +import org.apache.parquet.column.EncodingAccessor +import org.apache.parquet.column.page.DataPage +import org.apache.parquet.column.page.DataPageV1 +import org.apache.parquet.column.page.DataPageV2 +import org.apache.parquet.column.page.DictionaryPage +import org.apache.parquet.column.page.PageReader +import org.apache.parquet.column.values.RequiresPreviousReader +import org.apache.parquet.column.values.ValuesReader +import org.apache.parquet.io.ParquetDecodingException +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.apache.parquet.column.values.ValuesReader +import org.apache.parquet.column.Dictionary +import org.apache.parquet.column.Encoding +import org.apache.parquet.column.Dictionary + +object BaseColumnReader { + private val definitionLevelCacheSize = 128 + private val LOG: Log = Log.getLog(getClass) +} + +abstract class BaseColumnReader( + writerVersion: ParsedVersion, + path: ColumnDescriptor, + pageReader: PageReader, + val maxDefinitionLevel: Int, + val maxRepetitionLevel: Int) extends SpecificTypeGet { + + def columnName: String = path.getPath.mkString("/") + + private[this] lazy val totalValueCount = pageReader.getTotalValueCount + + protected lazy val dictionary = Option(pageReader.readDictionaryPage).map { dictionaryPage => + val dictionary = dictionaryPage.getEncoding.initDictionary(path, dictionaryPage) + dictionary + }.orNull + + import BaseColumnReader._ + + private[this] val repetitionLevels = new Array[Int](definitionLevelCacheSize) + private[this] val definitionLevels = new Array[Int](definitionLevelCacheSize) + + private[this] var levelIdx: Int = _ + private[this] var readValues: Int = _ + + private[this] var r_decoder: RunLengthBitPackingHybridDecoder = _ + private[this] var d_decoder: RunLengthBitPackingHybridDecoder = _ + + protected var currentEncoding: Encoding = _ + + private[this] var endOfPageValueCount: Long = 0L + private[this] var pageValueCount: Int = 0 + + protected var dataColumn: ValuesReader = _ + + private[this] var mutRecordPosition = -1L + + protected var dictionaryActive: Boolean = false + + // Tests if the target is behind the state + protected def isBehind(shouldBeAhead: Array[Int], shouldBeLessOrEqual: Array[Int]): Boolean = { + val minLen = if (shouldBeLessOrEqual.length < shouldBeAhead.length) shouldBeLessOrEqual.length else shouldBeAhead.length + var p = 0 + while (p < minLen) { + if (shouldBeLessOrEqual(p) > shouldBeAhead(p)) + return true + p += 1 + } + false + } + + @inline + protected final def recordPosition = mutRecordPosition + + def getRecordPosition = mutRecordPosition + + def getPrimitivePosition = readValues + + def advanceSetRecord(newRecordPosition: Long): Unit + + def primitiveAdvance(targetRecord: Long, targetOffset: Array[Int]): Boolean + + protected final def skip(): Unit = { + if (dataColumn == null) { + mutRecordPosition = 0L + loadData() + } else { + if (getCurrentDefinitionLevel == maxDefinitionLevel) + dataColumn.skip() + advanceDefRef() + } + } + + final def isFullyConsumed: Boolean = readValues >= totalValueCount + + protected final def getCurrentRepetitionLevel: Int = repetitionLevels(levelIdx) + + protected final def getCurrentDefinitionLevel: Int = definitionLevels(levelIdx) + + final def isPageFullyConsumed: Boolean = readValues >= endOfPageValueCount + + final def getTotalValueCount: Long = totalValueCount + + final def isNull: Boolean = getCurrentDefinitionLevel != maxDefinitionLevel + + protected def advanceDefRef(): Boolean = { + levelIdx += 1 + if (levelIdx > 0 && isFullyConsumed) sys.error("Consuming empty iterator") + + readValues += 1 + + val r = if (levelIdx >= definitionLevelCacheSize || isPageFullyConsumed) { + loadData + } else true + + if (getCurrentRepetitionLevel == 0) { + mutRecordPosition += 1L + } + r + } + + private[this] def updateDataAvailable(): Long = { + val pageRemaining: Long = endOfPageValueCount - readValues + if (pageRemaining > 0) + pageRemaining + else { + // If no data left we read the next page. + // to do this we will read in a new data column too, + // so must have advanced the data reader on before hand + // To call here we have to actually have consumed + // all the values aswell as the def/rep level info + // so its safe to just move on here + checkRead() + endOfPageValueCount - readValues + } + } + + private[this] def loadData(): Boolean = { + val pageRemaining = updateDataAvailable() + + if (pageRemaining > 0) { + if (r_decoder != null) + r_decoder.readInts(repetitionLevels) + if (d_decoder != null) + d_decoder.readInts(definitionLevels) + levelIdx = 0 + true + } else { + false + } + } + + private[this] def checkRead(): Unit = { + if (isPageFullyConsumed) { + if (isFullyConsumed) { + repetitionLevels(0) = 0 + definitionLevels(0) = 0 + levelIdx = 0 + } else + readPage() + } + } + + private[this] def readPage(): Unit = + pageReader.readPage() match { + case pageV1: DataPageV1 => readPageV1(pageV1) + case p => sys.error(s"unknown page type $p") + } + + var pageIdx = -1 + private[this] def readPageV1(page: DataPageV1) { + try { + levelIdx = 9999 // reset where we are in our cache of info from r_decoder and d_decoder + var offset: Int = 0 + pageIdx += 1 + println(s"Loading page $pageIdx for column $columnName") + val bytes = page.getBytes().toByteArray() + + val repetitionLevelEncoding = page.getRlEncoding + val definitionLevelEncoding = page.getRlEncoding + require(repetitionLevelEncoding == Encoding.RLE || repetitionLevelEncoding == Encoding.BIT_PACKED, s"repetitionLevelEncoding encoding should be the Hybrid runlength bit packing decoder, was ${repetitionLevelEncoding.toString}") + require(definitionLevelEncoding == Encoding.RLE || definitionLevelEncoding == Encoding.BIT_PACKED, s"definitionLevelEncoding encoding should be the Hybrid runlength bit packing decoder, was ${repetitionLevelEncoding.toString}") + + val r_bitWidth: Int = BytesUtils.getWidthFromMaxInt(EncodingAccessor.getMaxLevel(repetitionLevelEncoding, path, REPETITION_LEVEL)) + + if (r_bitWidth > 0) { + val r_in: ByteArrayInputStream = new ByteArrayInputStream(bytes, offset, bytes.length - offset) + val r_length: Int = BytesUtils.readIntLittleEndian(r_in) + offset += 4; // 4 is for the length which is stored as 4 bytes little endian + r_decoder = new RunLengthBitPackingHybridDecoder(r_bitWidth, bytes, offset, offset + r_length) + offset += r_length + } else { + r_decoder = null + } + + val d_bitWidth: Int = BytesUtils.getWidthFromMaxInt(EncodingAccessor.getMaxLevel(repetitionLevelEncoding, path, DEFINITION_LEVEL)) + + if (d_bitWidth > 0) { + val d_in: ByteArrayInputStream = new ByteArrayInputStream(bytes, offset, bytes.length - offset) + val d_length: Int = BytesUtils.readIntLittleEndian(d_in) + offset += 4; // 4 is for the length which is stored as 4 bytes little endian + d_decoder = new RunLengthBitPackingHybridDecoder(d_bitWidth, bytes, offset, offset + d_length) + offset += d_length + } else { + d_decoder = null + } + + if (DEBUG) LOG.debug("reading data at " + offset) + initDataReader(page.getValueEncoding(), bytes, offset, page.getValueCount()) + } catch { + case e: IOException => + throw new ParquetDecodingException("could not read page " + page + " in col " + path, e) + } + } + + private[this] def initDataReader(dataEncoding: Encoding, bytes: Array[Byte], offset: Int, valueCount: Int) { + this.currentEncoding = dataEncoding + this.pageValueCount = valueCount + this.endOfPageValueCount = readValues + pageValueCount + + dataColumn = dataEncoding.usesDictionary match { + case true if dictionary != null => + dictionaryActive = true + dataEncoding.getDictionaryBasedValuesReader(path, VALUES, dictionary) + case false => + dictionaryActive = false + dataEncoding.getValuesReader(path, VALUES) + case _ => sys.error("Tried to use dictionary encoding but have no dictionary") + } + + dataColumn.initFromPage(pageValueCount, bytes, offset) + } +} diff --git a/lui-core/src/main/scala/com/twitter/lui/column_reader/RunLengthBitPackingHybridDecoder.scala b/lui-core/src/main/scala/com/twitter/lui/column_reader/RunLengthBitPackingHybridDecoder.scala new file mode 100644 index 0000000000..aafcb5a8fa --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/column_reader/RunLengthBitPackingHybridDecoder.scala @@ -0,0 +1,190 @@ +/* + * 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 com.twitter.lui.column_reader + +import java.io.ByteArrayInputStream +import java.io.DataInputStream +import java.io.IOException + +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 + +/** + * Decodes values written in the grammar described in {@link RunLengthBitPackingHybridEncoder} + * + */ +object RunLengthBitPackingHybridDecoder { + private val LOG: Log = Log.getLog(classOf[RunLengthBitPackingHybridDecoder]) + private[RunLengthBitPackingHybridDecoder] sealed trait MODE + private[RunLengthBitPackingHybridDecoder] case object RLE extends MODE + private[RunLengthBitPackingHybridDecoder] case object PACKED extends MODE + private[RunLengthBitPackingHybridDecoder] case object COMPLETE extends MODE +} +class RunLengthBitPackingHybridDecoder(bitWidth: Int, backing: Array[Byte], var offset: Int, limit: Int) { + import RunLengthBitPackingHybridDecoder._ + Preconditions.checkArgument(bitWidth >= 0 && bitWidth <= 32, "bitWidth must be >= 0 and <= 32") + + private[this] val bytesWidth: Int = BytesUtils.paddedByteCountFromBits(bitWidth) + private[this] val packer: BytePacker = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth) + + private[this] var mode: MODE = RLE + private[this] var currentBuffer = new Array[Int](10) + private[this] var bufferLength = 10 + private[this] var readIdx = 0 + private[this] var currentCount = 0 + + def readInt(): Int = { + if (readIdx == currentCount) { + readNext() + } + mode match { + case RLE => + readIdx += 1 + currentBuffer(0) + case PACKED => + readIdx += 1 + currentBuffer(readIdx - 1) + case _ => + throw new ParquetDecodingException("EOF finished file") + } + } + + def readInts(target: Array[Int]): Unit = { + var pos = 0 + var rIdx = readIdx + var curCnt = currentCount + while (pos < target.length) { + if (rIdx == curCnt) { + readNext() + rIdx = 0 + curCnt = currentCount + if (curCnt == 0) // EOF + return + } + if (mode == RLE) { + while (pos < target.length && rIdx < curCnt) { + target(pos) = currentBuffer(0) + pos += 1 + rIdx += 1 + } + } else { + while (pos < target.length && rIdx < curCnt) { + target(pos) = currentBuffer(rIdx) + pos += 1 + rIdx += 1 + } + } + } + readIdx = rIdx + } + + private[this] def readUnsignedVarInt(): Int = { + var value = 0 + var i = 0 + var b = 0 + def updateB(): Int = { + b = backing(offset) + offset += 1 + b + } + + while ((updateB() & 0x80) != 0) { + value |= (b & 0x7F) << i + i += 7 + } + value | (b << i) + } + + private[this] def readIntLittleEndianPaddedOnBitWidth(): Int = { + val bw: Int = bytesWidth + var res = 0 + var o = offset + if (bw > 0) { + res += backing(o) & 0xff + o += 1 + if (bw > 1) { + res += (backing(o) & 0xff) << 8 + o += 1 + if (bw > 2) { + res += (backing(o) & 0xff) << 16 + o += 1 + if (bw > 3) { + res += (backing(o) & 0xff) << 24 + o += 1 + } + } + } + } + offset = o + res + } + + private[this] def growBuffer(requestedSize: Int): Unit = { + val oldLength = bufferLength + bufferLength = (requestedSize * 1.5).toInt + val oldBuffer = currentBuffer + currentBuffer = new Array[Int](bufferLength) + System.arraycopy(oldBuffer, 0, currentBuffer, 0, oldLength) + } + + private[this] final def readNext(): Unit = { + if (offset >= limit) { + currentCount = 0 + mode = COMPLETE + return + } + + val header = readUnsignedVarInt() + mode = if ((header & 1) == 0) RLE else PACKED + readIdx = 0 + mode match { + case RLE => + currentCount = header >>> 1 + currentBuffer(0) = readIntLittleEndianPaddedOnBitWidth() + case PACKED => + val numGroups = header >>> 1 + val nextCount = numGroups * 8 + var valueIndex = 0 + currentCount = nextCount + + if (bufferLength < currentCount) { + growBuffer(currentCount) + } + + // At the end of the file RLE data though, there might not be that many bytes left. + val tmp: Int = nextCount * bitWidth + val tmp2: Int = tmp / 8 + var bytesToRead: Int = if (tmp % 8 == 0) tmp2 else tmp2 + 1 + val remaining = limit - offset + bytesToRead = if (remaining < bytesToRead) remaining else bytesToRead + + while (valueIndex < currentCount) { + packer.unpack8Values(backing, offset, currentBuffer, valueIndex) + valueIndex += 8 + offset += bitWidth + } + case _ => + throw new ParquetDecodingException("not a valid mode " + mode) + } + } +} diff --git a/lui-core/src/main/scala/com/twitter/lui/column_reader/SpecificTypeGet.scala b/lui-core/src/main/scala/com/twitter/lui/column_reader/SpecificTypeGet.scala new file mode 100644 index 0000000000..147496d21c --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/column_reader/SpecificTypeGet.scala @@ -0,0 +1,37 @@ +package com.twitter.lui.column_reader +import java.nio.ByteBuffer + +trait HasCache { + protected def storeVToCache(position: Int): Unit + protected def cachePrimitivePosition: Array[Int] +} + +trait SpecificTypeGet { + def getInteger(): Int = { + throw new UnsupportedOperationException() + } + + def getBoolean(): Boolean = { + throw new UnsupportedOperationException() + } + + def getLong(): Long = { + throw new UnsupportedOperationException() + } + + def getBinary(): ByteBuffer = { + throw new UnsupportedOperationException() + } + + def getString(): String = { + throw new UnsupportedOperationException() + } + + def getFloat(): Float = { + throw new UnsupportedOperationException() + } + + def getDouble(): Double = { + throw new UnsupportedOperationException() + } +} \ No newline at end of file diff --git a/lui-core/src/main/scala/com/twitter/lui/column_reader/cache/CacheColumnReader.scala b/lui-core/src/main/scala/com/twitter/lui/column_reader/cache/CacheColumnReader.scala new file mode 100644 index 0000000000..3ee04b8597 --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/column_reader/cache/CacheColumnReader.scala @@ -0,0 +1,215 @@ +/* + * 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 com.twitter.lui.column_reader.cache + +import java.lang.String.format +import org.apache.parquet.Log.DEBUG +import org.apache.parquet.Preconditions.checkNotNull +import org.apache.parquet.column.ValuesType.DEFINITION_LEVEL +import org.apache.parquet.column.ValuesType.REPETITION_LEVEL +import org.apache.parquet.column.ValuesType.VALUES + +import java.io.ByteArrayInputStream +import java.io.IOException + +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 +import org.apache.parquet.column.ColumnDescriptor +import org.apache.parquet.column.Dictionary +import org.apache.parquet.column.EncodingAccessor +import org.apache.parquet.column.page.DataPage +import org.apache.parquet.column.page.DataPageV1 +import org.apache.parquet.column.page.DataPageV2 +import org.apache.parquet.column.page.DictionaryPage +import org.apache.parquet.column.page.PageReader +import org.apache.parquet.column.values.RequiresPreviousReader +import org.apache.parquet.column.values.ValuesReader +import org.apache.parquet.io.ParquetDecodingException +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.apache.parquet.column.values.ValuesReader +import org.apache.parquet.column.Dictionary +import org.apache.parquet.column.Encoding +import org.apache.parquet.column.Dictionary +import com.twitter.lui.column_reader._ + +object CacheColumnReader { + private val LOG: Log = Log.getLog(getClass) +} + +abstract class CacheColumnReader( + writerVersion: ParsedVersion, + path: ColumnDescriptor, + pageReader: PageReader, + maxDefinitionLevel: Int, + maxRepetitionLevel: Int) extends BaseColumnReader(writerVersion, path, pageReader, maxDefinitionLevel, maxRepetitionLevel) with SpecificTypeGet with HasCache { + + import CacheColumnReader._ + + protected var cacheRecordPosition = -1L + + protected var cacheSize: Int = 0 + + protected var cacheDef: Array[Int] = new Array[Int](8) + protected var cacheRep: Array[Int] = new Array[Int](8) + + private[this] def grow(cachePos: Int): Unit = { + if (cachePos >= cacheDef.length) { + val oldCacheDef = cacheDef + val oldCacheRep = cacheRep + val newSize = cacheDef.length * 2 + + cacheDef = new Array[Int](newSize) + cacheRep = new Array[Int](newSize) + + System.arraycopy(oldCacheDef, 0, cacheDef, 0, oldCacheDef.length) + System.arraycopy(oldCacheRep, 0, cacheRep, 0, oldCacheDef.length) + } + } + + def populateCache(): Unit = { + + cacheRecordPosition = recordPosition + var cacheIdx = 0 + while (recordPosition == cacheRecordPosition && !isFullyConsumed) { + grow(cacheIdx) + cacheDef(cacheIdx) = getCurrentDefinitionLevel + cacheRep(cacheIdx) = getCurrentRepetitionLevel + if (getCurrentDefinitionLevel == maxDefinitionLevel) { + storeVToCache(cacheIdx) + } + advanceDefRef() + cacheIdx += 1 + } + cacheSize = cacheIdx + } + + override def getRecordPosition = cacheRecordPosition + + def advanceSetRecord(newRecordPosition: Long): Unit = { + if (newRecordPosition != cacheRecordPosition) { + if (newRecordPosition < cacheRecordPosition) { + sys.error(s"$columnName -> Attempted to access an old field, requested record $newRecordPosition, while the cache contains $cacheRecordPosition") + } + + while (recordPosition < newRecordPosition) { + skip() + } + + if (!isFullyConsumed) { + populateCache() + cachePrimitivePosition(0) = 0 + var p = 0 + while (p < (maxRepetitionLevel + 1)) { + cachePrimitiveState(p) = 0 + p += 1 + } + } + } + } + + def notNull(definitionLevel: Int, recordIdx: Long): Boolean = { + advanceSetRecord(recordIdx) + cacheDef(0) >= definitionLevel + } + + def notNull(definitionLevel: Int, + recordIdx: Long, + cachePos: Array[Int], + stateOffsets: Array[Int], + targetOffsets: Array[Int]): Boolean = { + advanceSetRecord(recordIdx) + skipInsideRecord(definitionLevel, cachePos, stateOffsets, targetOffsets) + } + + protected var cachePrimitivePosition = Array(0) + var cachePrimitiveState: Array[Int] = (0 until (maxRepetitionLevel + 1)).map(_ => 0).toArray + + def primitiveAdvance(targetRecord: Long, targetOffset: Array[Int]): Boolean = { + advanceSetRecord(targetRecord) + if (!isBehind(targetOffset, cachePrimitiveState)) { + cachePrimitivePosition(0) = 0 + var p = 0 + while (p < cachePrimitiveState.length) { + cachePrimitiveState(p) = 0 + p += 1 + } + } + skipInsideRecord(maxDefinitionLevel, cachePrimitivePosition, cachePrimitiveState, targetOffset) + } + + def skipInsideRecord(definitionLevel: Int, previousPosition: Array[Int], stateOffset: Array[Int], targetOffset: Array[Int]): Boolean = { + + var p = 0 + val maxLen = (maxRepetitionLevel + 1) + var cachePosition: Int = previousPosition(0) + val cacheSize = this.cacheSize + + while (p < maxLen) { + if (stateOffset(p) > targetOffset(p)) { + previousPosition(0) = cachePosition + return false + } + + // the first one at any level won't have the flag set + while (stateOffset(p) < targetOffset(p)) { + // If we got onto a new record + // or the repetition level is such that we are looping further out than + // this advance calls for. + // The 0th one will occur on the first time, and is a red herring. + // we account for that by having the cache position be < the cache size + if (cachePosition >= cacheSize) { + previousPosition(0) = cachePosition + return false + } else { + cachePosition += 1 + if (cachePosition >= cacheSize) { + previousPosition(0) = cachePosition + return false + } + + val cRep = cacheRep(cachePosition) + + stateOffset(cRep) += 1 + var p2 = cRep + 1 + while (p2 < stateOffset.length) { + stateOffset(p2) = 0 + p2 += 1 + } + // if we are on an outer loop then we are done, nothing to find + if (cRep < p) { + previousPosition(0) = cachePosition + return false + } + } + } + p += 1; + } + // Now we have gotten to the best place we can, is it null? + previousPosition(0) = cachePosition + cacheDef(cachePosition) >= definitionLevel + } + +} diff --git a/lui-core/src/main/scala/com/twitter/lui/column_reader/cache/CacheColumnReaderImplementations.scala b/lui-core/src/main/scala/com/twitter/lui/column_reader/cache/CacheColumnReaderImplementations.scala new file mode 100644 index 0000000000..da47deb80e --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/column_reader/cache/CacheColumnReaderImplementations.scala @@ -0,0 +1,249 @@ +/* + * 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 com.twitter.lui.column_reader.cache + +import com.twitter.lui.column_reader._ +import java.lang.String.format +import org.apache.parquet.Log.DEBUG +import org.apache.parquet.Preconditions.checkNotNull +import org.apache.parquet.column.ValuesType.DEFINITION_LEVEL +import org.apache.parquet.column.ValuesType.REPETITION_LEVEL +import org.apache.parquet.column.ValuesType.VALUES +import java.nio.ByteBuffer + +import java.io.ByteArrayInputStream +import java.io.IOException + +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 +import org.apache.parquet.column.ColumnDescriptor +import org.apache.parquet.column.Dictionary +import org.apache.parquet.column.Encoding +import org.apache.parquet.column.page.DataPage +import org.apache.parquet.column.page.DataPageV1 +import org.apache.parquet.column.page.DataPageV2 +import org.apache.parquet.column.page.DictionaryPage +import org.apache.parquet.column.page.PageReader +import org.apache.parquet.column.values.RequiresPreviousReader +import org.apache.parquet.column.values.ValuesReader +import org.apache.parquet.io.ParquetDecodingException +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.apache.parquet.column.values.ValuesReader +import org.apache.parquet.column.Dictionary +import org.apache.parquet.column.Encoding +import org.apache.parquet.column.Dictionary + +final case class BooleanColumnReader( + writerVersion: ParsedVersion, + path: ColumnDescriptor, + pageReader: PageReader, + maxDefLvl: Int, + maxRepLvl: Int) extends CacheColumnReader(writerVersion, path, pageReader, maxDefLvl, maxRepLvl) { + private[this] lazy val dict: Array[Boolean] = if (dictionary == null) null else { + val maxId = dictionary.getMaxId + val dict = new Array[Boolean](maxId + 1) + var i = 0 + while (i <= maxId) { + dict(i) = dictionary.decodeToBoolean(i) + i = i + 1 + } + dict + } + + override def getBoolean(): Boolean = cacheV(cachePrimitivePosition(0)) + + private[this] var cacheV: Array[Boolean] = new Array[Boolean](8) + + private[this] def innerRead(): Boolean = + if (!dictionaryActive) + dataColumn.readBoolean + else { + dict(dataColumn.readValueDictionaryId) + } + + override def storeVToCache(position: Int): Unit = { + if (position >= cacheV.length) { + val oldCache = cacheV + cacheV = new Array[Boolean](scala.math.max(position, cacheV.length) * 2) + System.arraycopy(oldCache, 0, cacheV, 0, oldCache.length) + } + cacheV(position) = innerRead() + } +} + +final case class IntColumnReader( + writerVersion: ParsedVersion, + path: ColumnDescriptor, + pageReader: PageReader, + maxDefLvl: Int, + maxRepLvl: Int) extends CacheColumnReader(writerVersion, path, pageReader, maxDefLvl, maxRepLvl) { + private[this] lazy val dict: Array[Int] = if (dictionary == null) null else { + val maxId = dictionary.getMaxId + val dict = new Array[Int](maxId + 1) + var i = 0 + while (i <= maxId) { + dict(i) = dictionary.decodeToInt(i) + i = i + 1 + } + dict + } + + override def getInteger(): Int = cacheV(cachePrimitivePosition(0)) + + private[this] var cacheV: Array[Int] = new Array[Int](8) + + private[this] def innerRead(): Int = dataColumn.readInteger() + + override def storeVToCache(position: Int): Unit = { + if (position >= cacheV.length) { + val oldCache = cacheV + cacheV = new Array[Int](scala.math.max(position, cacheV.length) * 2) + System.arraycopy(oldCache, 0, cacheV, 0, oldCache.length) + } + cacheV(position) = innerRead() + } +} + +final case class LongColumnReader( + writerVersion: ParsedVersion, + path: ColumnDescriptor, + pageReader: PageReader, + maxDefLvl: Int, + maxRepLvl: Int) extends CacheColumnReader(writerVersion, path, pageReader, maxDefLvl, maxRepLvl) { + private[this] lazy val dict: Array[Long] = if (dictionary == null) null else { + val maxId = dictionary.getMaxId + val dict = new Array[Long](maxId + 1) + var i = 0 + while (i <= maxId) { + dict(i) = dictionary.decodeToLong(i) + i = i + 1 + } + dict + } + + override def getLong(): Long = cacheV(cachePrimitivePosition(0)) + + private[this] var cacheV: Array[Long] = new Array[Long](8) + + private[this] def innerRead(): Long = dataColumn.readLong() + + override def storeVToCache(position: Int): Unit = { + if (position >= cacheV.length) { + val oldCache = cacheV + cacheV = new Array[Long](scala.math.max(position, cacheV.length) * 2) + System.arraycopy(oldCache, 0, cacheV, 0, oldCache.length) + } + cacheV(position) = innerRead() + } +} + +final case class DoubleColumnReader( + writerVersion: ParsedVersion, + path: ColumnDescriptor, + pageReader: PageReader, + maxDefLvl: Int, + maxRepLvl: Int) extends CacheColumnReader(writerVersion, path, pageReader, maxDefLvl, maxRepLvl) { + private[this] lazy val dict: Array[Double] = if (dictionary == null) null else { + val maxId = dictionary.getMaxId + val dict = new Array[Double](maxId + 1) + var i = 0 + while (i <= maxId) { + dict(i) = dictionary.decodeToDouble(i) + i = i + 1 + } + dict + } + + override def getDouble(): Double = cacheV(cachePrimitivePosition(0)) + + private[this] var cacheV: Array[Double] = new Array[Double](8) + + private[this] def innerRead(): Double = dataColumn.readDouble() + + override def storeVToCache(position: Int): Unit = { + if (position >= cacheV.length) { + val oldCache = cacheV + cacheV = new Array[Double](scala.math.max(position, cacheV.length) * 2) + System.arraycopy(oldCache, 0, cacheV, 0, oldCache.length) + } + cacheV(position) = innerRead() + } +} + +case class StringBinaryContainer(b: Binary) { + lazy val toStr = b.toStringUsingUTF8 + lazy val toBB = b.toByteBuffer +} + +final case class StringColumnReader( + writerVersion: ParsedVersion, + path: ColumnDescriptor, + pageReader: PageReader, + maxDefLvl: Int, + maxRepLvl: Int) extends CacheColumnReader(writerVersion, path, pageReader, maxDefLvl, maxRepLvl) { + + private[this] lazy val dict: Array[StringBinaryContainer] = if (dictionary == null) null else { + val maxId = dictionary.getMaxId + val dict = new Array[StringBinaryContainer](maxId + 1) + var i = 0 + while (i <= maxId) { + dict(i) = StringBinaryContainer(dictionary.decodeToBinary(i)) + i = i + 1 + } + dict + } + + override def getString(): String = cacheV(cachePrimitivePosition(0)).toStr + override def getBinary(): ByteBuffer = cacheV(cachePrimitivePosition(0)).toBB + + private[this] var cacheV: Array[StringBinaryContainer] = new Array[StringBinaryContainer](8) + + private[this] def innerRead(): StringBinaryContainer = try { + if (!dictionaryActive) + StringBinaryContainer(dataColumn.readBytes) + else { + dict(dataColumn.readValueDictionaryId) + } + } catch { + case e: Throwable => + println(s""" + Attempted to read value about column: $columnName + Dictionary is:\n ${Option(dict).map(_.take(10).mkString("\n"))} + data column is : $dataColumn + data column uses dictionary: ${currentEncoding.usesDictionary} + """) + throw e + } + + override def storeVToCache(position: Int): Unit = { + if (position >= cacheV.length) { + val oldCache = cacheV + cacheV = new Array[StringBinaryContainer](scala.math.max(position, cacheV.length) * 2) + System.arraycopy(oldCache, 0, cacheV, 0, oldCache.length) + } + cacheV(position) = innerRead() + } +} diff --git a/lui-core/src/main/scala/com/twitter/lui/column_reader/noncache/NonCacheColumnReader.scala b/lui-core/src/main/scala/com/twitter/lui/column_reader/noncache/NonCacheColumnReader.scala new file mode 100644 index 0000000000..49e123959e --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/column_reader/noncache/NonCacheColumnReader.scala @@ -0,0 +1,93 @@ +/* + * 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 com.twitter.lui.column_reader.noncache + +import java.lang.String.format +import org.apache.parquet.Log.DEBUG +import org.apache.parquet.Preconditions.checkNotNull +import org.apache.parquet.column.ValuesType.DEFINITION_LEVEL +import org.apache.parquet.column.ValuesType.REPETITION_LEVEL +import org.apache.parquet.column.ValuesType.VALUES + +import java.io.ByteArrayInputStream +import java.io.IOException + +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 +import org.apache.parquet.column.ColumnDescriptor +import org.apache.parquet.column.Dictionary +import org.apache.parquet.column.EncodingAccessor +import org.apache.parquet.column.page.DataPage +import org.apache.parquet.column.page.DataPageV1 +import org.apache.parquet.column.page.DataPageV2 +import org.apache.parquet.column.page.DictionaryPage +import org.apache.parquet.column.page.PageReader +import org.apache.parquet.column.values.RequiresPreviousReader +import org.apache.parquet.column.values.ValuesReader +import org.apache.parquet.io.ParquetDecodingException +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.apache.parquet.column.values.ValuesReader +import org.apache.parquet.column.Dictionary +import org.apache.parquet.column.Encoding +import org.apache.parquet.column.Dictionary + +import com.twitter.lui.column_reader._ +object NonCacheColumnReader { + private val LOG: Log = Log.getLog(getClass) +} + +abstract class NonCacheColumnReader( + writerVersion: ParsedVersion, + path: ColumnDescriptor, + pageReader: PageReader, + maxDefinitionLevel: Int, + maxRepetitionLevel: Int) extends BaseColumnReader(writerVersion, path, pageReader, maxDefinitionLevel, maxRepetitionLevel) with SpecificTypeGet { + + import NonCacheColumnReader._ + + require(maxRepetitionLevel == 0, "The non cache class can only be used where we have no repetition's occuring") + + def advanceSetRecord(newRecordPosition: Long): Unit = { + if (newRecordPosition != recordPosition) { + while (recordPosition < newRecordPosition) { + skip() + } + } + } + + def primitiveAdvance(newRecordPosition: Long, targetOffset: Array[Int]): Boolean = { + while (recordPosition < newRecordPosition) { + skip() + } + val delta = newRecordPosition - recordPosition + if (delta < -1) sys.error(s"$columnName: \t Attempted to access an old field, trying to goto $newRecordPosition while on $recordPosition") + if (delta < 0) false else { + // Now we have gotten to the best place we can, is it null? + getCurrentDefinitionLevel == maxDefinitionLevel + } + } + +} diff --git a/lui-core/src/main/scala/com/twitter/lui/column_reader/noncache/NonCacheColumnReaderImplementations.scala b/lui-core/src/main/scala/com/twitter/lui/column_reader/noncache/NonCacheColumnReaderImplementations.scala new file mode 100644 index 0000000000..20b457ee3d --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/column_reader/noncache/NonCacheColumnReaderImplementations.scala @@ -0,0 +1,221 @@ +/* + * 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 com.twitter.lui.column_reader.noncache +import com.twitter.lui.column_reader._ +import java.lang.String.format +import org.apache.parquet.Log.DEBUG +import org.apache.parquet.Preconditions.checkNotNull +import org.apache.parquet.column.ValuesType.DEFINITION_LEVEL +import org.apache.parquet.column.ValuesType.REPETITION_LEVEL +import org.apache.parquet.column.ValuesType.VALUES +import java.nio.ByteBuffer +import java.io.ByteArrayInputStream +import java.io.IOException + +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 +import org.apache.parquet.column.ColumnDescriptor +import org.apache.parquet.column.Dictionary +import org.apache.parquet.column.Encoding +import org.apache.parquet.column.page.DataPage +import org.apache.parquet.column.page.DataPageV1 +import org.apache.parquet.column.page.DataPageV2 +import org.apache.parquet.column.page.DictionaryPage +import org.apache.parquet.column.page.PageReader +import org.apache.parquet.column.values.RequiresPreviousReader +import org.apache.parquet.column.values.ValuesReader +import org.apache.parquet.io.ParquetDecodingException +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.apache.parquet.column.values.ValuesReader +import org.apache.parquet.column.Dictionary +import org.apache.parquet.column.Encoding +import org.apache.parquet.column.Dictionary + +final case class BooleanColumnReader( + writerVersion: ParsedVersion, + path: ColumnDescriptor, + pageReader: PageReader, + maxDefLvl: Int, + maxRepLvl: Int) extends NonCacheColumnReader(writerVersion, path, pageReader, maxDefLvl, maxRepLvl) { + private[this] lazy val dict: Array[Boolean] = if (dictionary == null) null else { + val maxId = dictionary.getMaxId + val dict = new Array[Boolean](maxId + 1) + var i = 0 + while (i <= maxId) { + dict(i) = dictionary.decodeToBoolean(i) + i = i + 1 + } + dict + } + + override def getBoolean(): Boolean = { + val b = if (!dictionaryActive) + dataColumn.readBoolean + else { + dict(dataColumn.readValueDictionaryId) + } + advanceDefRef() + b + } +} + +final case class IntColumnReader( + writerVersion: ParsedVersion, + path: ColumnDescriptor, + pageReader: PageReader, + maxDefLvl: Int, + maxRepLvl: Int) extends NonCacheColumnReader(writerVersion, path, pageReader, maxDefLvl, maxRepLvl) { + private[this] lazy val dict: Array[Int] = if (dictionary == null) null else { + val maxId = dictionary.getMaxId + val dict = new Array[Int](maxId + 1) + var i = 0 + while (i <= maxId) { + dict(i) = dictionary.decodeToInt(i) + i = i + 1 + } + dict + } + + override def getInteger(): Int = { + val i = if (!dictionaryActive) + dataColumn.readInteger() + else { + dict(dataColumn.readValueDictionaryId) + } + advanceDefRef() + i + } + +} + +final case class LongColumnReader( + writerVersion: ParsedVersion, + path: ColumnDescriptor, + pageReader: PageReader, + maxDefLvl: Int, + maxRepLvl: Int) extends NonCacheColumnReader(writerVersion, path, pageReader, maxDefLvl, maxRepLvl) { + private[this] lazy val dict: Array[Long] = if (dictionary == null) null else { + val maxId = dictionary.getMaxId + val dict = new Array[Long](maxId + 1) + var i = 0 + while (i <= maxId) { + dict(i) = dictionary.decodeToLong(i) + i = i + 1 + } + dict + } + + override def getLong(): Long = { + val l = if (!dictionaryActive) + dataColumn.readLong() + else { + dict(dataColumn.readValueDictionaryId) + } + advanceDefRef() + l + } + +} + +final case class DoubleColumnReader( + writerVersion: ParsedVersion, + path: ColumnDescriptor, + pageReader: PageReader, + maxDefLvl: Int, + maxRepLvl: Int) extends NonCacheColumnReader(writerVersion, path, pageReader, maxDefLvl, maxRepLvl) { + private[this] lazy val dict: Array[Double] = if (dictionary == null) null else { + val maxId = dictionary.getMaxId + val dict = new Array[Double](maxId + 1) + var i = 0 + while (i <= maxId) { + dict(i) = dictionary.decodeToDouble(i) + i = i + 1 + } + dict + } + + override def getDouble(): Double = { + val d = if (!dictionaryActive) + dataColumn.readDouble() + else { + dict(dataColumn.readValueDictionaryId) + } + advanceDefRef() + d + } + +} + +final case class StringColumnReader( + writerVersion: ParsedVersion, + path: ColumnDescriptor, + pageReader: PageReader, + maxDefLvl: Int, + maxRepLvl: Int) extends NonCacheColumnReader(writerVersion, path, pageReader, maxDefLvl, maxRepLvl) { + + private[this] lazy val stringDict: Array[String] = if (dictionary == null) null else { + val maxId = dictionary.getMaxId + val dict = new Array[String](maxId + 1) + var i = 0 + while (i <= maxId) { + dict(i) = dictionary.decodeToBinary(i).toStringUsingUTF8 + i = i + 1 + } + dict + } + + private[this] lazy val bbDict: Array[ByteBuffer] = if (dictionary == null) null else { + val maxId = dictionary.getMaxId + val dict = new Array[ByteBuffer](maxId + 1) + var i = 0 + while (i <= maxId) { + dict(i) = dictionary.decodeToBinary(i).toByteBuffer + i = i + 1 + } + dict + } + + override def getString(): String = { + val s = if (!dictionaryActive) + dataColumn.readBytes.toStringUsingUTF8 + else { + stringDict(dataColumn.readValueDictionaryId) + } + advanceDefRef() + s + } + + override def getBinary(): ByteBuffer = { + val s = if (!dictionaryActive) + dataColumn.readBytes.toByteBuffer + else { + bbDict(dataColumn.readValueDictionaryId) + } + advanceDefRef() + s + } + +} diff --git a/lui-core/src/main/scala/com/twitter/lui/hadoop/ReadContext.scala b/lui-core/src/main/scala/com/twitter/lui/hadoop/ReadContext.scala new file mode 100644 index 0000000000..18831b136e --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/hadoop/ReadContext.scala @@ -0,0 +1,28 @@ +/* + * 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 com.twitter.lui.hadoop + +import org.apache.hadoop.conf.Configuration + +import org.apache.parquet.io.api.RecordMaterializer +import org.apache.parquet.schema.MessageType +import org.apache.parquet.schema.MessageTypeParser + +case class ReadContext(requestedSchema: MessageType, + readSupportMetadata: Map[String, String] = Map()) diff --git a/lui-core/src/main/scala/com/twitter/lui/hadoop/ReadSupport.scala b/lui-core/src/main/scala/com/twitter/lui/hadoop/ReadSupport.scala new file mode 100644 index 0000000000..0261c12372 --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/hadoop/ReadSupport.scala @@ -0,0 +1,89 @@ +/* + * 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 com.twitter.lui.hadoop + +import java.util.{ Map => JMap } + +import org.apache.hadoop.conf.Configuration + +import org.apache.parquet.io.api.RecordMaterializer +import org.apache.parquet.schema.MessageType +import org.apache.parquet.schema.MessageTypeParser +import org.apache.parquet.hadoop.api.InitContext +import com.twitter.lui.RecordReader +import org.apache.parquet.column.page.PageReadStore +import scala.reflect.ClassTag + +object ReadSupport { + + /** + * configuration key for a parquet read projection schema + */ + val PARQUET_READ_SCHEMA: String = "parquet.read.schema" + + /** + * attempts to validate and construct a {@link MessageType} from a read projection schema + * + * @param fileMessageType the typed schema of the source + * @param partialReadSchemaString the requested projection schema + * @return the typed schema that should be used to read + */ + def getSchemaForRead(fileMessageType: MessageType, partialReadSchemaString: String): MessageType = { + if (partialReadSchemaString == null) + fileMessageType + else { + val requestedMessageType: MessageType = MessageTypeParser.parseMessageType(partialReadSchemaString) + getSchemaForRead(fileMessageType, requestedMessageType) + } + } + + def getSchemaForRead(fileMessageType: MessageType, projectedMessageType: MessageType): MessageType = { + fileMessageType.checkContains(projectedMessageType) + projectedMessageType + } + +} +abstract class ReadSupport[T] { + + private[this] var classTagHolder: ClassTag[T] = _ + + protected final def classTag: ClassTag[T] = classTagHolder + + private[lui] def setClassTag(ct: ClassTag[T]) { + classTagHolder = ct + } + + def getRecordReader( + columns: PageReadStore, + configuration: Configuration, + createdBy: String, + fileMetadata: Map[String, String], + fileSchema: MessageType, + readContext: ReadContext, + strictTypeChecking: Boolean): RecordReader[T] + + /** + * called in {@link org.apache.hadoop.mapreduce.InputFormat#getSplits(org.apache.hadoop.mapreduce.JobContext)} in the front end + * + * @param context the initialisation context + * @return the readContext that defines how to read the file + */ + def init(context: InitContext): ReadContext + +} diff --git a/lui-core/src/main/scala/com/twitter/lui/inputformat/ClientSideMetadataSplitStrategy.scala b/lui-core/src/main/scala/com/twitter/lui/inputformat/ClientSideMetadataSplitStrategy.scala new file mode 100644 index 0000000000..259c05e2f4 --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/inputformat/ClientSideMetadataSplitStrategy.scala @@ -0,0 +1,171 @@ +package com.twitter.lui.inputformat + +import java.lang.Boolean.TRUE +import org.apache.parquet.Preconditions.checkArgument + +import java.io.IOException +import java.util.ArrayList +import java.util.Arrays +import java.util.Collection +import java.util.Collections +import java.util.Comparator +import java.util.HashMap +import java.util.HashSet +import java.util.{ List => JList, Map => JMap, Set => JSet } + +import org.apache.hadoop.conf.Configurable +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.BlockLocation +import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.PathFilter +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapreduce.InputSplit +import org.apache.hadoop.mapreduce.Job +import org.apache.hadoop.mapreduce.JobContext +import org.apache.hadoop.mapreduce.RecordReader +import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat + +import org.apache.hadoop.mapreduce.lib.input.FileSplit +import org.apache.parquet.Log +import org.apache.parquet.Preconditions +import org.apache.parquet.filter.UnboundRecordFilter +import org.apache.parquet.filter2.compat.FilterCompat +import org.apache.parquet.filter2.compat.FilterCompat.Filter +import org.apache.parquet.filter2.compat.RowGroupFilter +import org.apache.parquet.filter2.predicate.FilterPredicate +import org.apache.parquet.hadoop.api.InitContext +import com.twitter.lui.hadoop.ReadSupport +import com.twitter.lui.hadoop.ReadContext +import org.apache.parquet.hadoop.metadata.BlockMetaData +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData +import org.apache.parquet.hadoop.metadata.ParquetMetadata +import org.apache.parquet.hadoop.util.ConfigurationUtil +import org.apache.parquet.hadoop.util.ContextUtil +import org.apache.parquet.hadoop.util.HiddenFileFilter +import org.apache.parquet.hadoop.util.SerializationUtil +import org.apache.parquet.io.ParquetDecodingException +import org.apache.parquet.schema.MessageType +import org.apache.parquet.schema.MessageTypeParser +import org.apache.parquet.hadoop.ParquetInputSplit +import scala.collection.JavaConverters._ +import org.apache.parquet.hadoop.Footer + +object ClientSideMetadataSplitStrategy { + + /** + * groups together all the data blocks for the same HDFS block + * + * @param rowGroupBlocks data blocks (row groups) + * @param hdfsBlocksArray hdfs blocks + * @param fileStatus the containing file + * @param requestedSchema the schema requested by the user + * @param readSupportMetadata the metadata provided by the readSupport implementation in init + * @param minSplitSize the mapred.min.split.size + * @param maxSplitSize the mapred.max.split.size + * @return the splits (one per HDFS block) + * @throws IOException If hosts can't be retrieved for the HDFS block + */ + def generateSplits[T]( + rowGroupBlocks: JList[BlockMetaData], + hdfsBlocksArray: Array[BlockLocation], + fileStatus: FileStatus, + requestedSchema: String, + readSupportMetadata: Map[String, String], minSplitSize: Long, maxSplitSize: Long): JList[ParquetInputSplit] = { + + val splitRowGroups: JList[SplitInfo] = + generateSplitInfo(rowGroupBlocks, hdfsBlocksArray, minSplitSize, maxSplitSize) + + //generate splits from rowGroups of each split + val resultSplits: JList[ParquetInputSplit] = new ArrayList[ParquetInputSplit]() + splitRowGroups.asScala.foreach{ splitInfo => + val split: ParquetInputSplit = splitInfo.getParquetInputSplit(fileStatus, requestedSchema, readSupportMetadata) + resultSplits.add(split) + } + resultSplits + } + + def generateSplitInfo( + rowGroupBlocks: JList[BlockMetaData], + hdfsBlocksArray: Array[BlockLocation], + minSplitSize: Long, maxSplitSize: Long): JList[SplitInfo] = { + + if (maxSplitSize < minSplitSize || maxSplitSize < 0 || minSplitSize < 0) { + throw new ParquetDecodingException("maxSplitSize and minSplitSize should be positive and max should be greater or equal to the minSplitSize: maxSplitSize = " + maxSplitSize + "; minSplitSize is " + minSplitSize) + } + val hdfsBlocks: HDFSBlocks = HDFSBlocks(hdfsBlocksArray) + hdfsBlocks.checkBelongingToANewHDFSBlock(rowGroupBlocks.get(0)) + var currentSplit: SplitInfo = new SplitInfo(hdfsBlocks.getCurrentBlock) + + //assign rowGroups to splits + var splitRowGroups: JList[SplitInfo] = new ArrayList[SplitInfo]() + checkSorted(rowGroupBlocks); //assert row groups are sorted + rowGroupBlocks.asScala.foreach{ rowGroupMetadata => + if ((hdfsBlocks.checkBelongingToANewHDFSBlock(rowGroupMetadata) + && currentSplit.getCompressedByteSize >= minSplitSize + && currentSplit.getCompressedByteSize > 0) + || currentSplit.getCompressedByteSize >= maxSplitSize) { + //create a new split + splitRowGroups.add(currentSplit); //finish previous split + currentSplit = new SplitInfo(hdfsBlocks.getCurrentBlock) + } + currentSplit.addRowGroup(rowGroupMetadata) + } + + if (currentSplit.getRowGroupCount > 0) { + splitRowGroups.add(currentSplit) + } + + splitRowGroups + } + + private def checkSorted(rowGroupBlocks: JList[BlockMetaData]) { + var previousOffset = 0L + rowGroupBlocks.asScala.foreach { rowGroup => + val currentOffset: Long = rowGroup.getStartingPos + if (currentOffset < previousOffset) { + throw new ParquetDecodingException("row groups are not sorted: previous row groups starts at " + previousOffset + ", current row group starts at " + currentOffset) + } + } + } + + private val LOG: Log = Log.getLog(getClass) + +} +class ClientSideMetadataSplitStrategy { + import ClientSideMetadataSplitStrategy._ + + def getSplits(configuration: Configuration, footers: JList[Footer], + maxSplitSize: Long, minSplitSize: Long, readContext: ReadContext): JList[ParquetInputSplit] = { + val splits: JList[ParquetInputSplit] = new ArrayList[ParquetInputSplit]() + + var totalRowGroups: Long = 0L + + footers.asScala.foreach{ footer => + val file: Path = footer.getFile + LOG.debug(file) + val fs: FileSystem = file.getFileSystem(configuration) + val fileStatus: FileStatus = fs.getFileStatus(file) + val parquetMetaData: ParquetMetadata = footer.getParquetMetadata() + val blocks: JList[BlockMetaData] = parquetMetaData.getBlocks() + + totalRowGroups += blocks.size() + + val fileBlockLocations: Array[BlockLocation] = fs.getFileBlockLocations(fileStatus, 0, fileStatus.getLen()) + splits.addAll( + generateSplits( + blocks, + fileBlockLocations, + fileStatus, + readContext.requestedSchema.toString(), + readContext.readSupportMetadata, + minSplitSize, + maxSplitSize)) + } + + splits + } + +} diff --git a/lui-core/src/main/scala/com/twitter/lui/inputformat/ColumnChunkPageReadStore.scala b/lui-core/src/main/scala/com/twitter/lui/inputformat/ColumnChunkPageReadStore.scala new file mode 100644 index 0000000000..f6b1214980 --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/inputformat/ColumnChunkPageReadStore.scala @@ -0,0 +1,63 @@ +/* + * 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 com.twitter.lui.inputformat + +import java.io.IOException + +import org.apache.parquet.Ints +import org.apache.parquet.Log +import org.apache.parquet.column.ColumnDescriptor +import org.apache.parquet.column.page.DataPage +import org.apache.parquet.column.page.DataPageV1 +import org.apache.parquet.column.page.DictionaryPage +import org.apache.parquet.column.page.PageReadStore +import org.apache.parquet.column.page.PageReader +import org.apache.parquet.io.ParquetDecodingException + +/** + * TODO: should this actually be called RowGroupImpl or something? + * The name is kind of confusing since it references three different "entities" + * in our format: columns, chunks, and pages + * + */ +private[inputformat] object ColumnChunkPageReadStore { + private val LOG: Log = Log.getLog(classOf[ColumnChunkPageReadStore]) +} + +private[inputformat] class ColumnChunkPageReadStore(rowCount: Long) extends PageReadStore { + import ColumnChunkPageReadStore._ + + private[this] var readers = Map[ColumnDescriptor, ColumnChunkPageReader]() + + override def getRowCount(): Long = rowCount + + override def getPageReader(path: ColumnDescriptor): PageReader = + + readers.get(path).getOrElse { + throw new IllegalArgumentException(s"$path is not in the store: ${readers.keys.mkString(",")} for $rowCount") + } + + def addColumn(path: ColumnDescriptor, reader: ColumnChunkPageReader) { + if (readers.contains(path)) { + throw new RuntimeException(path + " was added twice") + } + readers = readers + (path -> reader) + } + +} diff --git a/lui-core/src/main/scala/com/twitter/lui/inputformat/ColumnChunkPageReader.scala b/lui-core/src/main/scala/com/twitter/lui/inputformat/ColumnChunkPageReader.scala new file mode 100644 index 0000000000..b86353fb4d --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/inputformat/ColumnChunkPageReader.scala @@ -0,0 +1,179 @@ +/* + * 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 com.twitter.lui.inputformat + +import com.twitter.lui.inputformat.codec._ +import java.io.ByteArrayInputStream +import java.io.IOException +import org.apache.hadoop.fs.FSDataInputStream +import org.apache.parquet.bytes.BytesInput +import org.apache.parquet.column.ColumnDescriptor +import org.apache.parquet.column.page.DataPage +import org.apache.parquet.column.page.DataPageV1 +import org.apache.parquet.column.page.DictionaryPage +import org.apache.parquet.column.page.PageReader +import org.apache.parquet.column.page.PageReadStore +import org.apache.parquet.format.converter.ParquetMetadataConverter +import org.apache.parquet.format.converter.ParquetMetadataConverter.fromParquetStatistics +import org.apache.parquet.format.DataPageHeader +import org.apache.parquet.format.DictionaryPageHeader +import org.apache.parquet.format.PageHeader +import org.apache.parquet.format.PageType +import org.apache.parquet.format.Util +import org.apache.parquet.Ints +import org.apache.parquet.io.ParquetDecodingException +import org.apache.parquet.Log + +class CustomByteArrayInputStream(data: Array[Byte]) extends ByteArrayInputStream(data) { + def getPos = pos + def readPageHeader(): PageHeader = Util.readPageHeader(this) + def readAsBytesInput(size: Int): BytesInput = { + val r: BytesInput = BytesInput.from(buf, pos, size) + pos += size + r + } +} + +case class ColumnChunkPreStaged(descriptor: ChunkDescriptor, f: FSDataInputStream, codecFactory: CodecFactory, createdBy: String) + +case class ColumnChunkData(var pagesInChunk: List[DataPage], + dictionaryPage: DictionaryPage, + decompressor: BytesDecompressor) { + def popDataPage: DataPage = if (pagesInChunk.isEmpty) null else { + val h = pagesInChunk.head + pagesInChunk = pagesInChunk.tail + h + } +} + +object ColumnChunkData { + private val converter: ParquetMetadataConverter = new ParquetMetadataConverter() + private val LOG: Log = Log.getLog(classOf[ColumnChunkData]) + + def apply(preStaged: ColumnChunkPreStaged): ColumnChunkData = { + val mc = preStaged.descriptor.metadata + preStaged.f.seek(mc.getStartingPos) + val totalSize = mc.getTotalSize.asInstanceOf[Int] + val chunksBytes: Array[Byte] = new Array[Byte](totalSize) + preStaged.f.readFully(chunksBytes) + + val bais = new CustomByteArrayInputStream(chunksBytes) + val descriptor = preStaged.descriptor + val createdBy = preStaged.createdBy + val decompressor: BytesDecompressor = preStaged.codecFactory.getDecompressor(descriptor.metadata.getCodec) + + var pagesInChunk: List[DataPage] = List[DataPage]() + var dictionaryPage: DictionaryPage = null + var valuesCountReadSoFar: Long = 0 + + while (valuesCountReadSoFar < descriptor.metadata.getValueCount) { + val pageHeader: PageHeader = bais.readPageHeader + val uncompressedPageSize: Int = pageHeader.getUncompressed_page_size + val compressedPageSize: Int = pageHeader.getCompressed_page_size + pageHeader.getType match { + case PageType.DICTIONARY_PAGE => + // there is only one dictionary page per column chunk + if (dictionaryPage != null) { + throw new ParquetDecodingException(s"more than one dictionary page in column : ${descriptor.col}") + } + val dicHeader: DictionaryPageHeader = pageHeader.getDictionary_page_header + dictionaryPage = new DictionaryPage( + bais.readAsBytesInput(compressedPageSize), + uncompressedPageSize, + dicHeader.getNum_values, + converter.getEncoding(dicHeader.getEncoding)) + case PageType.DATA_PAGE => + val dataHeaderV1: DataPageHeader = pageHeader.getData_page_header + pagesInChunk = pagesInChunk :+ new DataPageV1( + bais.readAsBytesInput(compressedPageSize), + dataHeaderV1.getNum_values, + uncompressedPageSize, + fromParquetStatistics(createdBy, dataHeaderV1.getStatistics, descriptor.col.getType), + converter.getEncoding(dataHeaderV1.getRepetition_level_encoding), + converter.getEncoding(dataHeaderV1.getDefinition_level_encoding), + converter.getEncoding(dataHeaderV1.getEncoding)) + valuesCountReadSoFar += dataHeaderV1.getNum_values + + case other => + LOG.warn(s"skipping page of type ${pageHeader.getType} of size ${compressedPageSize}") + bais.skip(compressedPageSize) + } + } + if (valuesCountReadSoFar != descriptor.metadata.getValueCount) { + throw new IOException( + s"""Expected ${descriptor.metadata.getValueCount} values in column chunk at + | offset ${descriptor.metadata.getFirstDataPageOffset} but got + | $valuesCountReadSoFar values instead over ${pagesInChunk.size} + | pages ending at file offset ${descriptor.fileOffset + bais.getPos}""".stripMargin('|')) + } + ColumnChunkData(pagesInChunk, + dictionaryPage, + decompressor) + } +} + +private[inputformat] class ColumnChunkPageReader(prestagedData: ColumnChunkPreStaged) extends PageReader { + + lazy val columnChunkData: ColumnChunkData = ColumnChunkData(prestagedData) + + private[this] lazy val valueCount = columnChunkData.pagesInChunk.map(_.getValueCount).sum + + override def getTotalValueCount: Long = valueCount + + override def readPage(): DataPage = + if (columnChunkData.pagesInChunk.isEmpty) { + null + } else { + val compressedPage: DataPage = columnChunkData.popDataPage + + compressedPage match { + case dataPageV1: DataPageV1 => + try { + new DataPageV1( + columnChunkData.decompressor.decompress(dataPageV1.getBytes(), dataPageV1.getUncompressedSize()), + dataPageV1.getValueCount(), + dataPageV1.getUncompressedSize(), + dataPageV1.getStatistics(), + dataPageV1.getRlEncoding(), + dataPageV1.getDlEncoding(), + dataPageV1.getValueEncoding()) + } catch { + case e: IOException => + throw new ParquetDecodingException("could not decompress page", e) + } + } + } + + override def readDictionaryPage(): DictionaryPage = + if (columnChunkData.dictionaryPage == null) { + null + } else { + try { + val compressedDictionaryPage = columnChunkData.dictionaryPage + new DictionaryPage( + columnChunkData.decompressor.decompress(compressedDictionaryPage.getBytes(), compressedDictionaryPage.getUncompressedSize()), + compressedDictionaryPage.getDictionarySize(), + compressedDictionaryPage.getEncoding()) + } catch { + + case e: IOException => + throw new RuntimeException(e) + } + } +} \ No newline at end of file diff --git a/lui-core/src/main/scala/com/twitter/lui/inputformat/FileStatusWrapper.scala b/lui-core/src/main/scala/com/twitter/lui/inputformat/FileStatusWrapper.scala new file mode 100644 index 0000000000..d78cee989f --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/inputformat/FileStatusWrapper.scala @@ -0,0 +1,87 @@ +/* + * 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 com.twitter.lui.inputformat + +import java.lang.Boolean.TRUE +import org.apache.parquet.Preconditions.checkArgument + +import java.io.IOException +import java.util.ArrayList +import java.util.Arrays +import java.util.Collection +import java.util.Collections +import java.util.Comparator +import java.util.HashMap +import java.util.HashSet +import java.util.{ List => JList, Map => JMap, Set => JSet } + +import org.apache.hadoop.conf.Configurable +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.BlockLocation +import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.PathFilter +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapreduce.InputSplit +import org.apache.hadoop.mapreduce.Job +import org.apache.hadoop.mapreduce.JobContext +import org.apache.hadoop.mapreduce.RecordReader +import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat + +import org.apache.hadoop.mapreduce.lib.input.FileSplit +import org.apache.parquet.Log +import org.apache.parquet.Preconditions +import org.apache.parquet.filter.UnboundRecordFilter +import org.apache.parquet.filter2.compat.FilterCompat +import org.apache.parquet.filter2.compat.FilterCompat.Filter +import org.apache.parquet.filter2.compat.RowGroupFilter +import org.apache.parquet.filter2.predicate.FilterPredicate +import org.apache.parquet.hadoop.api.InitContext +import com.twitter.lui.hadoop.ReadSupport +import com.twitter.lui.hadoop.ReadContext +import org.apache.parquet.hadoop.metadata.BlockMetaData +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData +import org.apache.parquet.hadoop.metadata.GlobalMetaData +import org.apache.parquet.hadoop.metadata.ParquetMetadata +import org.apache.parquet.hadoop.util.ConfigurationUtil +import org.apache.parquet.hadoop.util.ContextUtil +import org.apache.parquet.hadoop.util.HiddenFileFilter +import org.apache.parquet.hadoop.util.SerializationUtil +import org.apache.parquet.io.ParquetDecodingException +import org.apache.parquet.schema.MessageType +import org.apache.parquet.schema.MessageTypeParser +import org.apache.parquet.hadoop.ParquetInputSplit +import org.apache.parquet.hadoop.BadConfigurationException +import scala.collection.JavaConverters._ +import org.apache.parquet.hadoop.Footer + +case class FileStatusWrapper(fileStatus: FileStatus) { + def getModificationTime: Long = fileStatus.getModificationTime + override def hashCode = fileStatus.hashCode + + override def equals(other: Any): Boolean = + other match { + case fsW: FileStatusWrapper if fsW.fileStatus.equals(fileStatus) => true + case _ => false + } + + override def toString: String = fileStatus.getPath.toString +} \ No newline at end of file diff --git a/lui-core/src/main/scala/com/twitter/lui/inputformat/HDFSBlocks.scala b/lui-core/src/main/scala/com/twitter/lui/inputformat/HDFSBlocks.scala new file mode 100644 index 0000000000..531b9ff165 --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/inputformat/HDFSBlocks.scala @@ -0,0 +1,110 @@ +package com.twitter.lui.inputformat + +import java.lang.Boolean.TRUE +import org.apache.parquet.Preconditions.checkArgument + +import java.io.IOException +import java.util.ArrayList +import java.util.Arrays +import java.util.Collection +import java.util.Collections +import java.util.Comparator +import java.util.HashMap +import java.util.HashSet +import java.util.{ List => JList, Map => JMap, Set => JSet } + +import org.apache.hadoop.conf.Configurable +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.BlockLocation +import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.PathFilter +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapreduce.InputSplit +import org.apache.hadoop.mapreduce.Job +import org.apache.hadoop.mapreduce.JobContext +import org.apache.hadoop.mapreduce.RecordReader +import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat + +import org.apache.hadoop.mapreduce.lib.input.FileSplit +import org.apache.parquet.Log +import org.apache.parquet.Preconditions +import org.apache.parquet.filter.UnboundRecordFilter +import org.apache.parquet.filter2.compat.FilterCompat +import org.apache.parquet.filter2.compat.FilterCompat.Filter +import org.apache.parquet.filter2.compat.RowGroupFilter +import org.apache.parquet.filter2.predicate.FilterPredicate +import org.apache.parquet.hadoop.api.InitContext +import com.twitter.lui.hadoop.ReadSupport +import com.twitter.lui.hadoop.ReadContext +import org.apache.parquet.hadoop.metadata.BlockMetaData +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData +import org.apache.parquet.hadoop.metadata.ParquetMetadata +import org.apache.parquet.hadoop.util.ConfigurationUtil +import org.apache.parquet.hadoop.util.ContextUtil +import org.apache.parquet.hadoop.util.HiddenFileFilter +import org.apache.parquet.hadoop.util.SerializationUtil +import org.apache.parquet.io.ParquetDecodingException +import org.apache.parquet.schema.MessageType +import org.apache.parquet.schema.MessageTypeParser +import org.apache.parquet.hadoop.ParquetInputSplit +import scala.collection.JavaConverters._ + +//Wrapper of hdfs blocks, keep track of which HDFS block is being used + +private[inputformat] object HDFSBlocks { + + def apply(hdfsBlocks: Array[BlockLocation]): HDFSBlocks = { + val comparator = new Comparator[BlockLocation]() { + override def compare(b1: BlockLocation, b2: BlockLocation): Int = + java.lang.Long.signum(b1.getOffset() - b2.getOffset()) + } + Arrays.sort(hdfsBlocks, comparator) + new HDFSBlocks(hdfsBlocks) + } +} + +private[inputformat] class HDFSBlocks private (hdfsBlocks: Array[BlockLocation]) { + var currentStartHdfsBlockIndex: Int = 0; //the hdfs block index corresponding to the start of a row group + var currentMidPointHDFSBlockIndex: Int = 0; // the hdfs block index corresponding to the mid-point of a row group, a split might be created only when the midpoint of the rowgroup enters a new hdfs block + + private[this] def getHDFSBlockEndingPosition(hdfsBlockIndex: Int): Long = { + val hdfsBlock = hdfsBlocks(hdfsBlockIndex) + hdfsBlock.getOffset + hdfsBlock.getLength - 1 + } + + /** + * @param rowGroupMetadata + * @return true if the mid point of row group is in a new hdfs block, and also move the currentHDFSBlock pointer to the correct index that contains the row group + * return false if the mid point of row group is in the same hdfs block + */ + def checkBelongingToANewHDFSBlock(rowGroupMetadata: BlockMetaData): Boolean = { + var isNewHdfsBlock: Boolean = false + val rowGroupMidPoint: Long = rowGroupMetadata.getStartingPos() + (rowGroupMetadata.getCompressedSize() / 2) + + //if mid point is not in the current HDFS block any more, return true + while (rowGroupMidPoint > getHDFSBlockEndingPosition(currentMidPointHDFSBlockIndex)) { + isNewHdfsBlock = true + currentMidPointHDFSBlockIndex += 1 + if (currentMidPointHDFSBlockIndex >= hdfsBlocks.length) + throw new ParquetDecodingException("the row group is not in hdfs blocks in the file: midpoint of row groups is " + + rowGroupMidPoint + + ", the end of the hdfs block is " + + getHDFSBlockEndingPosition(currentMidPointHDFSBlockIndex - 1)) + } + + while (rowGroupMetadata.getStartingPos() > getHDFSBlockEndingPosition(currentStartHdfsBlockIndex)) { + currentStartHdfsBlockIndex += 1 + if (currentStartHdfsBlockIndex >= hdfsBlocks.length) + throw new ParquetDecodingException("The row group does not start in this file: row group offset is " + + rowGroupMetadata.getStartingPos() + + " but the end of hdfs blocks of file is " + + getHDFSBlockEndingPosition(currentStartHdfsBlockIndex)) + } + isNewHdfsBlock + } + + def getCurrentBlock: BlockLocation = hdfsBlocks(currentStartHdfsBlockIndex) +} \ No newline at end of file diff --git a/lui-core/src/main/scala/com/twitter/lui/inputformat/LruCache.scala b/lui-core/src/main/scala/com/twitter/lui/inputformat/LruCache.scala new file mode 100644 index 0000000000..58dde20cfd --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/inputformat/LruCache.scala @@ -0,0 +1,156 @@ +/* + * 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 com.twitter.lui.inputformat + +import org.apache.parquet.Log + +import java.util.LinkedHashMap +import java.util.Map + +/** + * A basic implementation of an LRU cache. Besides evicting the least recently + * used entries (either based on insertion or access order), this class also + * checks for "stale" entries as entries are inserted or retrieved (note + * "staleness" is defined by the entries themselves (see + * {@link org.apache.parquet.hadoop.LruCache.Value}). + * + * @param The key type. Acts as the key in a {@link java.util.LinkedHashMap} + * @param The value type. Must extend {@link org.apache.parquet.hadoop.LruCache.Value} + * so that the "staleness" of the value can be easily determined. + */ +object LruCache { + private val LOG: Log = Log.getLog(classOf[LruCache[_, _]]) + + private val DEFAULT_LOAD_FACTOR = 0.75f + def apply[K, V <: LruCache.Value[K, V]](maxSize: Int)(implicit ev: Null <:< V) = new LruCache[K, V](maxSize, DEFAULT_LOAD_FACTOR, true) + + /** + * {@link org.apache.parquet.hadoop.LruCache} expects all values to follow this + * interface so the cache can determine 1) whether values are current (e.g. + * the referenced data has not been modified/updated in such a way that the + * value is no longer useful) and 2) whether a value is strictly "newer" + * than another value. + * + * @param The key type. + * @param Provides a bound for the {@link #isNewerThan(V)} method + */ + trait Value[K, V] { + /** + * Is the value still current (e.g. has the referenced data been + * modified/updated in such a way that the value is no longer useful) + * @param key the key associated with this value + * @return {@code true} the value is still current, {@code false} the value + * is no longer useful + */ + def isCurrent(key: K): Boolean + + /** + * Compares this value with the specified value to check for relative age. + * @param otherValue the value to be compared. + * @return {@code true} the value is strictly newer than the other value, + * {@code false} the value is older or just + * as new as the other value. + */ + def isNewerThan(otherValue: V): Boolean + } + +} +final class LruCache[K, V <: LruCache.Value[K, V]](maxSize: Int, loadFactor: Float, accessOrder: Boolean)(implicit ev: Null <:< V) { + import LruCache._ + + private[this] val cacheMap: LinkedHashMap[K, V] = { + val initialCapacity = Math.round(maxSize / loadFactor) + new LinkedHashMap[K, V](initialCapacity, loadFactor, accessOrder) { + + override def removeEldestEntry(eldest: Map.Entry[K, V]): Boolean = + size > maxSize + + } + } + + /** + * Removes the mapping for the specified key from this cache if present. + * @param key key whose mapping is to be removed from the cache + * @return the previous value associated with key, or null if there was no + * mapping for key. + */ + def remove(key: K): V = + cacheMap.remove(key) + + /** + * Associates the specified value with the specified key in this cache. The + * value is only inserted if it is not null and it is considered current. If + * the cache previously contained a mapping for the key, the old value is + * replaced only if the new value is "newer" than the old one. + * @param key key with which the specified value is to be associated + * @param newValue value to be associated with the specified key + */ + def put(key: K, newValue: V) { + if (newValue == null || !newValue.isCurrent(key)) { + if (Log.WARN) { + LOG.warn("Ignoring new cache entry for '" + key + "' because it is " + + (if (newValue == null) "null" else "not current")) + } + return + } + + val oldValue: V = cacheMap.get(key) + if (oldValue != null && oldValue.isNewerThan(newValue)) { + if (Log.WARN) { + LOG.warn("Ignoring new cache entry for '" + key + "' because " + + "existing cache entry is newer") + } + return + } + + // no existing value or new value is newer than old value + cacheMap.put(key, newValue) + } + + /** + * Removes all of the mappings from this cache. The cache will be empty + * after this call returns. + */ + def clear(): Unit = { + cacheMap.clear() + } + + /** + * Returns the value to which the specified key is mapped, or null if 1) the + * value is not current or 2) this cache contains no mapping for the key. + * @param key the key whose associated value is to be returned + * @return the value to which the specified key is mapped, or null if 1) the + * value is not current or 2) this cache contains no mapping for the key + */ + def getCurrentValue(key: K): V = { + val value = cacheMap.get(key) + if (value != null && !value.isCurrent(key)) { + // value is not current; remove it and return null + remove(key) + ev(null) + } else value + } + + /** + * Returns the number of key-value mappings in this cache. + * @return the number of key-value mappings in this cache. + */ + def size = cacheMap.size + +} diff --git a/lui-core/src/main/scala/com/twitter/lui/inputformat/MapRedParquetInputFormat.scala b/lui-core/src/main/scala/com/twitter/lui/inputformat/MapRedParquetInputFormat.scala new file mode 100644 index 0000000000..d33667524f --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/inputformat/MapRedParquetInputFormat.scala @@ -0,0 +1,178 @@ +/* + * 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 com.twitter.lui.inputformat + +import java.lang.Boolean.TRUE +import java.util.Arrays.asList + +import java.io.DataInput +import java.io.DataOutput +import java.io.IOException +import java.util.{ List => JList, Map => JMap, Set => JSet } + +import org.apache.hadoop.mapred.FileSplit +import org.apache.hadoop.mapred.InputSplit +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapred.RecordReader +import org.apache.hadoop.mapred.Reporter + +import org.apache.parquet.hadoop.Footer +import org.apache.parquet.hadoop.ParquetInputSplit +import scala.collection.JavaConverters._ +import org.apache.parquet.hadoop.mapred.Container + +object MapRedParquetInputFormat { + + private[MapRedParquetInputFormat] class ParquetInputSplitWrapper(var realSplit: ParquetInputSplit) extends InputSplit { + + def this() = this(null) + + override def getLength: Long = realSplit.getLength + + override def getLocations: Array[String] = realSplit.getLocations + + override def readFields(in: DataInput) { + realSplit = new ParquetInputSplit() + realSplit.readFields(in) + } + + override def write(out: DataOutput) { + realSplit.write(out) + } + } + + private[MapRedParquetInputFormat] object RecordReaderWrapper { + def apply[V](oldSplit: InputSplit, oldJobConf: JobConf, reporter: Reporter): RecordReaderWrapper[V] = + try { + val realReader = new ParquetRecordReader[V](ParquetInputFormat.getReadSupportInstance[V](oldJobConf)) + + oldSplit match { + case pisq: ParquetInputSplitWrapper => realReader.initialize(pisq.realSplit, oldJobConf, reporter) + case fs: FileSplit => realReader.initialize(fs, oldJobConf, reporter) + case _ => + throw new IllegalArgumentException("Invalid split (not a FileSplit or ParquetInputSplitWrapper): " + oldSplit) + } + + // read once to gain access to key and value objects + val (firstRecord, valueContainer, eof) = if (realReader.nextKeyValue()) { + val valueContainer = new Container[V]() + valueContainer.set(realReader.getCurrentValue()) + (true, valueContainer, false) + } else { + (false, null, true) + } + + RecordReaderWrapper[V]( + realReader, + oldSplit.getLength, + valueContainer, + firstRecord, + eof) + + } catch { + case e: InterruptedException => + Thread.interrupted() + throw new IOException(e) + } + } + + private[MapRedParquetInputFormat] case class RecordReaderWrapper[V] private ( + realReader: ParquetRecordReader[V], + splitLen: Long, + private val valueContainer: Container[V], + private var firstRecord: Boolean, + private var eof: Boolean) extends RecordReader[Void, Container[V]] { + + override def close() { + realReader.close() + } + + override def createKey(): Void = + null + + override def createValue: Container[V] = + valueContainer + + override def getPos: Long = + (splitLen * getProgress()).toLong + + override def getProgress: Float = + try { + realReader.getProgress + } catch { + case e: InterruptedException => + Thread.interrupted() + throw new IOException(e) + } + + override def next(key: Void, value: Container[V]): Boolean = { + if (eof) { + return false + } + + if (firstRecord) { // key & value are already read. + firstRecord = false + return true + } + + try { + if (realReader.nextKeyValue()) { + if (value != null) value.set(realReader.getCurrentValue()) + return true + } + } catch { + case e: InterruptedException => + throw new IOException(e) + } + + eof = true; // strictly not required, just for consistency + return false + } + } +} + +class MapRedParquetInputFormat[V] extends org.apache.hadoop.mapred.FileInputFormat[Void, Container[V]] { + import MapRedParquetInputFormat._ + protected val realInputFormat: ParquetInputFormat[V] = new ParquetInputFormat[V]() + + override def getRecordReader(split: InputSplit, job: JobConf, + reporter: Reporter): RecordReader[Void, Container[V]] = + RecordReaderWrapper[V](split, job, reporter) + + override def getSplits(job: JobConf, numSplits: Int): Array[InputSplit] = { + if (isTaskSideMetaData(job)) { + return super.getSplits(job, numSplits) + } + + val footers: JList[Footer] = getFooters(job) + val splits: JList[ParquetInputSplit] = realInputFormat.getSplits(job, footers) + if (splits == null) { + return null + } + splits.asScala.map { split: ParquetInputSplit => + new ParquetInputSplitWrapper(split) + }.toArray + } + + def getFooters(job: JobConf): JList[Footer] = + realInputFormat.getFooters(job, super.listStatus(job).toList.asJava) + + def isTaskSideMetaData(job: JobConf): Boolean = + job.getBoolean(ParquetInputFormat.TASK_SIDE_METADATA, TRUE) +} diff --git a/lui-core/src/main/scala/com/twitter/lui/inputformat/ParquetFileReader.scala b/lui-core/src/main/scala/com/twitter/lui/inputformat/ParquetFileReader.scala new file mode 100644 index 0000000000..40ed48c74a --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/inputformat/ParquetFileReader.scala @@ -0,0 +1,139 @@ +/* + * 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 com.twitter.lui.inputformat + +import org.apache.parquet.bytes.BytesUtils.readIntLittleEndian +import org.apache.parquet.format.converter.ParquetMetadataConverter.fromParquetStatistics + +import java.io.ByteArrayInputStream +import java.io.Closeable +import java.io.IOException +import java.io.SequenceInputStream +import java.util.Arrays +import java.util.Collection +import java.util.Collections +import java.util.concurrent.Callable +import java.util.concurrent.ExecutionException +import java.util.concurrent.ExecutorService +import java.util.concurrent.Executors +import java.util.concurrent.Future + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FSDataInputStream +import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path + +import org.apache.parquet.Log +import org.apache.parquet.bytes.BytesInput +import org.apache.parquet.column.ColumnDescriptor +import org.apache.parquet.column.page.DataPage +import org.apache.parquet.column.page.DataPageV1 +import org.apache.parquet.column.page.DataPageV2 +import org.apache.parquet.column.page.DictionaryPage +import org.apache.parquet.column.page.PageReadStore +import org.apache.parquet.hadoop.metadata.ColumnPath +import org.apache.parquet.format.DataPageHeader +import org.apache.parquet.format.DataPageHeaderV2 +import org.apache.parquet.format.DictionaryPageHeader +import org.apache.parquet.format.PageHeader +import org.apache.parquet.format.Util +import org.apache.parquet.format.converter.ParquetMetadataConverter +import org.apache.parquet.format.converter.ParquetMetadataConverter.MetadataFilter +import org.apache.parquet.hadoop.metadata.BlockMetaData +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData +import org.apache.parquet.hadoop.metadata.FileMetaData +import org.apache.parquet.hadoop.metadata.ParquetMetadata +import org.apache.parquet.hadoop.util.HiddenFileFilter +import org.apache.parquet.io.ParquetDecodingException +import org.apache.parquet.hadoop.metadata.FileMetaData +import com.twitter.lui.inputformat.codec.{ CodecFactory, BytesDecompressor } +import scala.collection.JavaConverters._ + +import scala.collection.mutable.{ Map => MMap } + +/** + * information needed to read a column chunk + */ +private[inputformat] case class ChunkDescriptor( + col: ColumnDescriptor, + metadata: ColumnChunkMetaData, + fileOffset: Long, + size: Int) + +object ParquetFileReader { + private val LOG: Log = Log.getLog(classOf[ParquetFileReader]) + private val converter: ParquetMetadataConverter = new ParquetMetadataConverter() + +} + +class ParquetFileReader(configuration: Configuration, + fileMetadata: FileMetaData, + filePath: Path, + blocks: IndexedSeq[BlockMetaData], + columns: List[ColumnDescriptor]) extends Closeable { + import ParquetFileReader._ + + val createdBy = if (fileMetadata == null) null else fileMetadata.getCreatedBy + + private[this] val codecFactory: CodecFactory = new CodecFactory(configuration) + + private[this] val f: FSDataInputStream = { + val fs: FileSystem = filePath.getFileSystem(configuration) + fs.open(filePath) + } + + private[this] val paths: Map[ColumnPath, ColumnDescriptor] = columns.map { col => + ColumnPath.get(col.getPath: _*) -> col + }.toMap + + private[this] var currentBlock: Int = 0 + + /** + * Reads all the columns requested from the row group at the current file position. + * @throws IOException if an error occurs while reading + * @return the PageReadStore which can provide PageReaders for each column. + */ + def readNextRowGroup(): PageReadStore = { + if (currentBlock == blocks.size) { + null + } else { + val block: BlockMetaData = blocks(currentBlock) + if (block.getRowCount == 0) { + throw new RuntimeException("Illegal row group of 0 rows") + } + + val columnChunkPageReadStore: ColumnChunkPageReadStore = new ColumnChunkPageReadStore(block.getRowCount) + + block.getColumns.asScala.foreach { mc => + val columnDescriptor = paths(mc.getPath) + val descriptor = ChunkDescriptor(columnDescriptor, mc, mc.getStartingPos, mc.getTotalSize.asInstanceOf[Int]) + + columnChunkPageReadStore.addColumn(descriptor.col, new ColumnChunkPageReader(ColumnChunkPreStaged(descriptor, f, codecFactory, createdBy))) + } + currentBlock += 1 + columnChunkPageReadStore + } + } + + override def close() { + f.close() + codecFactory.release() + } +} diff --git a/lui-core/src/main/scala/com/twitter/lui/inputformat/ParquetInputFormat.scala b/lui-core/src/main/scala/com/twitter/lui/inputformat/ParquetInputFormat.scala new file mode 100644 index 0000000000..c8cadf95d7 --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/inputformat/ParquetInputFormat.scala @@ -0,0 +1,384 @@ +/* + * 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 com.twitter.lui.inputformat + +import java.lang.Boolean.TRUE +import org.apache.parquet.Preconditions.checkArgument + +import java.io.IOException +import java.util.ArrayList +import java.util.Arrays +import java.util.Collection +import java.util.Collections +import java.util.Comparator +import java.util.HashMap +import java.util.HashSet +import java.util.{ List => JList, Map => JMap, Set => JSet } + +import org.apache.hadoop.conf.Configurable +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.BlockLocation +import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.PathFilter +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapreduce.InputSplit +import org.apache.hadoop.mapreduce.Job +import org.apache.hadoop.mapreduce.JobContext +import org.apache.hadoop.mapreduce.RecordReader +import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat + +import org.apache.hadoop.mapreduce.lib.input.FileSplit +import org.apache.parquet.Log +import org.apache.parquet.Preconditions +import org.apache.parquet.filter.UnboundRecordFilter +import org.apache.parquet.filter2.compat.FilterCompat +import org.apache.parquet.filter2.compat.FilterCompat.Filter +import org.apache.parquet.filter2.compat.RowGroupFilter +import org.apache.parquet.filter2.predicate.FilterPredicate +import org.apache.parquet.hadoop.api.InitContext +import com.twitter.lui.hadoop.ReadSupport +import com.twitter.lui.hadoop.ReadContext +import org.apache.parquet.hadoop.metadata.BlockMetaData +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData +import org.apache.parquet.hadoop.metadata.GlobalMetaData +import org.apache.parquet.hadoop.metadata.ParquetMetadata +import org.apache.parquet.hadoop.util.ConfigurationUtil +import org.apache.parquet.hadoop.util.ContextUtil +import org.apache.parquet.hadoop.util.HiddenFileFilter +import org.apache.parquet.hadoop.util.SerializationUtil +import org.apache.parquet.io.ParquetDecodingException +import org.apache.parquet.schema.MessageType +import org.apache.parquet.schema.MessageTypeParser +import org.apache.parquet.hadoop.{ ParquetInputSplit, ParquetInputSplitBridge } +import org.apache.parquet.hadoop.BadConfigurationException +import scala.collection.JavaConverters._ +import org.apache.parquet.hadoop.Footer +import scala.reflect.ClassTag +/** + * The input format to read a Parquet file. + * + * It requires an implementation of {@link ReadSupport} to materialize the records. + * + * The requestedSchema will control how the original records get projected by the loader. + * It must be a subset of the original schema. Only the columns needed to reconstruct the records with the requestedSchema will be scanned. + * + * @see #READ_SUPPORT_CLASS + * @see #UNBOUND_RECORD_FILTER + * @see #STRICT_TYPE_CHECKING + * @see #FILTER_PREDICATE + * @see #TASK_SIDE_METADATA + * + * @param [T] the type of the materialized records + */ +object ParquetInputFormat { + val LOG: Log = Log.getLog(getClass) + + /** + * key to configure the ReadSupport implementation + */ + val READ_SUPPORT_CLASS = "parquet.read.support.class" + + val TARGET_CLASS = "parquet.target.class" + + /** + * key to configure the filter + */ + val UNBOUND_RECORD_FILTER = "parquet.read.filter" + + /** + * key to configure type checking for conflicting schemas (default: true) + */ + val STRICT_TYPE_CHECKING = "parquet.strict.typing" + + /** + * key to configure the filter predicate + */ + val FILTER_PREDICATE = "parquet.private.read.filter.predicate" + + /** + * key to turn on or off task side metadata loading (default true) + * if true then metadata is read on the task side and some tasks may finish immediately. + * if false metadata is read on the client which is slower if there is a lot of metadata but tasks will only be spawn if there is work to do. + */ + val TASK_SIDE_METADATA = "parquet.task.side.metadata" + + /** + * key to turn off file splitting. See PARQUET-246. + */ + val SPLIT_FILES = "parquet.split.files" + + private val MIN_FOOTER_CACHE_SIZE = 100 + + def setTaskSideMetaData(job: Job, taskSideMetadata: Boolean) { + ContextUtil.getConfiguration(job).setBoolean(TASK_SIDE_METADATA, taskSideMetadata) + } + + def isTaskSideMetaData(configuration: Configuration): Boolean = + configuration.getBoolean(TASK_SIDE_METADATA, TRUE) + + def setReadSupportClass(conf: JobConf, readSupportClass: Class[_], targetClass: Class[_]) { + conf.set(READ_SUPPORT_CLASS, readSupportClass.getName) + conf.set(TARGET_CLASS, targetClass.getName) + } + + def getClassFromConfig(configuration: Configuration, configName: String, assignableFromOpt: Option[Class[_]]): (String, Class[_]) = { + val className = configuration.get(configName) + if (className == null) { + return null; + } + + try { + val foundClass = configuration.getClassByName(className) + assignableFromOpt.foreach { assignableFrom => + if (!assignableFrom.isAssignableFrom(foundClass)) { + throw new BadConfigurationException("class " + className + " set in job conf at " + + configName + " is not a subclass of " + assignableFrom.getCanonicalName()); + } + } + (className, foundClass) + } catch { + case e: ClassNotFoundException => + throw new BadConfigurationException("could not instantiate class " + className + " set in job conf at " + configName, e); + } + } + + def getReadSupportClass[T](configuration: Configuration): (String, Class[_ <: ReadSupport[T]]) = { + val (name, clazz) = getClassFromConfig(configuration, READ_SUPPORT_CLASS, Some(classOf[ReadSupport[_]])) + + (name, clazz.asInstanceOf[Class[ReadSupport[T]]]) + } + + def getTargetClass[T](configuration: Configuration): (String, Class[T]) = { + val (name, clazz) = getClassFromConfig(configuration, TARGET_CLASS, None) + + (name, clazz.asInstanceOf[Class[T]]) + } + + def getReadSupportInstance[T](configuration: Configuration): ReadSupport[T] = { + val (className, readSupportClass) = getReadSupportClass[T](configuration) + try { + val inst = readSupportClass.newInstance + + val (targetClassName, clazz) = getTargetClass(configuration) + inst.setClassTag(ClassTag(clazz)) + inst + + } catch { + case e: InstantiationException => throw new BadConfigurationException(s"could not instantiate read support class: $className", e) + case e: IllegalAccessException => throw new BadConfigurationException(s"could not instantiate read support class: $className", e) + } + } + + private def getAllFileRecursively( + files: JList[FileStatus], conf: Configuration): JList[FileStatus] = { + val result: JList[FileStatus] = new ArrayList[FileStatus]() + + files.asScala.foreach { file => + if (file.isDir()) { + val p: Path = file.getPath() + val fs: FileSystem = p.getFileSystem(conf) + staticAddInputPathRecursively(result, fs, p, HiddenFileFilter.INSTANCE) + } else { + result.add(file) + } + } + LOG.info("Total input paths to process : " + result.size()) + result + } + + private def staticAddInputPathRecursively(result: JList[FileStatus], + fs: FileSystem, path: Path, inputFilter: PathFilter): Unit = { + fs.listStatus(path, inputFilter).foreach { stat: FileStatus => + if (stat.isDir()) { + staticAddInputPathRecursively(result, fs, stat.getPath(), inputFilter) + } else { + result.add(stat) + } + } + } + + object FootersCacheValue { + def apply(status: FileStatusWrapper, footer: Footer): FootersCacheValue = + new FootersCacheValue(status.getModificationTime, new Footer(footer.getFile, footer.getParquetMetadata)) + } + class FootersCacheValue(val modificationTime: Long, val footer: Footer) extends LruCache.Value[FileStatusWrapper, FootersCacheValue] { + + override def isCurrent(key: FileStatusWrapper): Boolean = { + val currentModTime: Long = key.getModificationTime + val isCurrent: Boolean = modificationTime >= currentModTime + if (Log.DEBUG && !isCurrent) { + LOG.debug("The cache value for '" + key + "' is not current: " + + "cached modification time=" + modificationTime + ", " + + "current modification time: " + currentModTime) + } + isCurrent + } + + def getFooter: Footer = footer + + override def isNewerThan(otherValue: FootersCacheValue): Boolean = + otherValue == null || modificationTime > otherValue.modificationTime + + def getPath: Path = footer.getFile + } +} + +class ParquetInputFormat[T] extends FileInputFormat[Void, T] { + import ParquetInputFormat._ + + private[this] var footersCache: LruCache[FileStatusWrapper, FootersCacheValue] = _ + + /** + * {@inheritDoc} + */ + + override def createRecordReader( + inputSplit: InputSplit, + taskAttemptContext: TaskAttemptContext): RecordReader[Void, T] = { + val conf: Configuration = ContextUtil.getConfiguration(taskAttemptContext) + val readSupport: ReadSupport[T] = getReadSupportInstance[T](conf) + ParquetRecordReader[T](readSupport) + } + + protected override def isSplitable(context: JobContext, filename: Path) = + ContextUtil.getConfiguration(context).getBoolean(SPLIT_FILES, true) + + override def getSplits(jobContext: JobContext): JList[InputSplit] = { + val configuration: Configuration = ContextUtil.getConfiguration(jobContext) + val splits: JList[InputSplit] = new ArrayList[InputSplit]() + + if (isTaskSideMetaData(configuration)) { + // Although not required by the API, some clients may depend on always + // receiving ParquetInputSplit. Translation is required at some point. + super.getSplits(jobContext).asScala.foreach{ split => + Preconditions.checkArgument(split.isInstanceOf[FileSplit], + "Cannot wrap non-FileSplit: " + split) + splits.add(ParquetInputSplitBridge.from(split.asInstanceOf[FileSplit])) + } + return splits + } else { + splits.addAll(getSplits(configuration, getFooters(jobContext))) + } + + splits + } + + def getSplits(configuration: Configuration, footers: JList[Footer]): JList[ParquetInputSplit] = { + val strictTypeChecking = configuration.getBoolean(STRICT_TYPE_CHECKING, true) + val maxSplitSize = configuration.getLong("mapred.max.split.size", java.lang.Long.MAX_VALUE) + val minSplitSize = Math.max(getFormatMinSplitSize(), configuration.getLong("mapred.min.split.size", 0L)) + if (maxSplitSize < 0 || minSplitSize < 0) { + throw new ParquetDecodingException("maxSplitSize or minSplitSize should not be negative: maxSplitSize = " + maxSplitSize + "; minSplitSize = " + minSplitSize); + } + val globalMetaData: GlobalMetaData = org.apache.parquet.hadoop.ParquetFileWriterBridge.getGlobalMetaData(footers, strictTypeChecking) + val readContext: ReadContext = getReadSupportInstance(configuration).init(new InitContext( + configuration, + globalMetaData.getKeyValueMetaData(), + globalMetaData.getSchema())) + + new ClientSideMetadataSplitStrategy().getSplits(configuration, footers, maxSplitSize, minSplitSize, readContext) + } + + /* + * This is to support multi-level/recursive directory listing until + * MAPREDUCE-1577 is fixed. + */ + override protected def listStatus(jobContext: JobContext): JList[FileStatus] = + getAllFileRecursively(super.listStatus(jobContext), ContextUtil.getConfiguration(jobContext)) + + /** + * @param jobContext the current job context + * @return the footers for the files + * @throws IOException + */ + def getFooters(jobContext: JobContext): JList[Footer] = { + val statuses: JList[FileStatus] = listStatus(jobContext) + if (statuses.isEmpty()) { + return Collections.emptyList() + } + val config: Configuration = ContextUtil.getConfiguration(jobContext) + val footers: JList[Footer] = new ArrayList[Footer](statuses.size()) + val missingStatuses: JSet[FileStatus] = new HashSet[FileStatus]() + val missingStatusesMap: JMap[Path, FileStatusWrapper] = + new HashMap[Path, FileStatusWrapper](missingStatuses.size()) + + if (footersCache == null) { + footersCache = LruCache[FileStatusWrapper, FootersCacheValue](Math.max(statuses.size(), MIN_FOOTER_CACHE_SIZE)) + } + statuses.asScala.foreach { status => + val statusWrapper: FileStatusWrapper = new FileStatusWrapper(status) + val cacheEntry: FootersCacheValue = footersCache.getCurrentValue(statusWrapper) + if (Log.DEBUG) { + LOG.debug(s"""Cache entry ${if (cacheEntry == null) "not" else ""} found for '${status.getPath}'""") + } + if (cacheEntry != null) { + footers.add(cacheEntry.getFooter) + } else { + missingStatuses.add(status) + missingStatusesMap.put(status.getPath, statusWrapper) + } + } + + if (Log.DEBUG) { + LOG.debug("found " + footers.size() + " footers in cache and adding up " + + "to " + missingStatuses.size() + " missing footers to the cache") + } + + if (missingStatuses.isEmpty()) { + return footers + } + + val newFooters: JList[Footer] = getFooters(config, missingStatuses) + newFooters.asScala.foreach { newFooter => + // Use the original file status objects to make sure we store a + // conservative (older) modification time (i.e. in case the files and + // footers were modified and it's not clear which version of the footers + // we have) + val fileStatus: FileStatusWrapper = missingStatusesMap.get(newFooter.getFile()) + footersCache.put(fileStatus, FootersCacheValue(fileStatus, newFooter)) + } + + footers.addAll(newFooters) + return footers + } + + /** + * the footers for the files + * @param configuration to connect to the file system + * @param statuses the files to open + * @return the footers of the files + * @throws IOException + */ + def getFooters(configuration: Configuration, statuses: Collection[FileStatus]): JList[Footer] = { + if (Log.DEBUG) LOG.debug("reading " + statuses.size() + " files") + val taskSideMetaData = isTaskSideMetaData(configuration) + org.apache.parquet.hadoop.ParquetFileReader.readAllFootersInParallelUsingSummaryFiles(configuration, statuses, taskSideMetaData) + } + + /** + * @param jobContext the current job context + * @return the merged metadata from the footers + * @throws IOException + */ + def getGlobalMetaData(jobContext: JobContext): GlobalMetaData = + org.apache.parquet.hadoop.ParquetFileWriterBridge.getGlobalMetaData(getFooters(jobContext)) +} \ No newline at end of file diff --git a/lui-core/src/main/scala/com/twitter/lui/inputformat/ParquetRecordReader.scala b/lui-core/src/main/scala/com/twitter/lui/inputformat/ParquetRecordReader.scala new file mode 100644 index 0000000000..1fb06d88c2 --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/inputformat/ParquetRecordReader.scala @@ -0,0 +1,192 @@ +/** + * 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 com.twitter.lui.inputformat + +import org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER +import org.apache.parquet.format.converter.ParquetMetadataConverter.range +import org.apache.parquet.hadoop.ParquetFileReader.readFooter +import com.twitter.lui.inputformat.ParquetInputFormat.SPLIT_FILES + +import java.io.IOException +import java.util.ArrayList +import java.util.Arrays +import java.util.{ List => JList, Map => JMap, Set => JSet, HashSet } + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapred.Reporter +import org.apache.hadoop.mapreduce.InputSplit +import org.apache.hadoop.mapreduce.RecordReader +import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.hadoop.mapreduce.TaskInputOutputContext + +import org.apache.hadoop.mapreduce.lib.input.FileSplit +import org.apache.parquet.CorruptDeltaByteArrays +import org.apache.parquet.Log +import org.apache.parquet.column.Encoding +import org.apache.parquet.filter.UnboundRecordFilter +import org.apache.parquet.filter2.compat.FilterCompat +import org.apache.parquet.filter2.compat.FilterCompat.Filter +import com.twitter.lui.hadoop.ReadSupport +import org.apache.parquet.hadoop.metadata.BlockMetaData +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData +import org.apache.parquet.hadoop.metadata.FileMetaData +import org.apache.parquet.hadoop.metadata.ParquetMetadata +import org.apache.parquet.hadoop.util.ContextUtil +import org.apache.parquet.hadoop.util.counters.BenchmarkCounter +import org.apache.parquet.io.ParquetDecodingException +import org.apache.parquet.schema.MessageType +import com.twitter.lui.record_reader.InternalParquetRecordReader +import org.apache.parquet.hadoop.{ ParquetInputSplit, ParquetInputSplitBridge } +import scala.collection.JavaConverters._ + +/** + * Reads the records from a block of a Parquet file + * + * @see ParquetInputFormat + * + * @author Julien Le Dem + * + * @param [T] type of the materialized records + */ +object ParquetRecordReader { + private val LOG: Log = Log.getLog(getClass) + +} +case class ParquetRecordReader[T](readSupport: ReadSupport[T]) extends RecordReader[Void, T] { + + var internalReader: InternalParquetRecordReader[T] = _ + import ParquetRecordReader._ + + override def close() { + internalReader.close() + } + + /** + * always returns null + */ + override def getCurrentKey(): Void = + null + + override def getCurrentValue(): T = internalReader.getCurrentValue() + + override def getProgress(): Float = internalReader.getProgress + + override def initialize(inputSplit: InputSplit, context: TaskAttemptContext) { + context match { + case taskContext: TaskInputOutputContext[_, _, _, _] => BenchmarkCounter.initCounterFromContext(taskContext) + case _ => + LOG.error("Can not initialize counter due to context is not a instance of TaskInputOutputContext, but is " + + context.getClass().getCanonicalName()) + } + + initializeInternalReader(toParquetSplit(inputSplit), ContextUtil.getConfiguration(context)) + } + + def initialize(inputSplit: InputSplit, configuration: Configuration, reporter: Reporter) { + BenchmarkCounter.initCounterFromReporter(reporter, configuration) + initializeInternalReader(toParquetSplit(inputSplit), configuration) + } + + private[this] def initializeInternalReader(split: ParquetInputSplit, configuration: Configuration) { + val path: Path = split.getPath() + val rowGroupOffsets: Array[Long] = split.getRowGroupOffsets + // if task.side.metadata is set, rowGroupOffsets is null + val (blocks: IndexedSeq[BlockMetaData], footer: ParquetMetadata) = if (rowGroupOffsets == null) { + // then we need to apply the predicate push down filter + val footer = readFooter(configuration, path, range(split.getStart, split.getEnd)) + (footer.getBlocks.asScala.toIndexedSeq, footer) + } else { + // otherwise we find the row groups that were selected on the client + val footer = readFooter(configuration, path, NO_FILTER) + val offsets: JSet[Long] = new HashSet[Long]() + rowGroupOffsets.foreach{ offset => + offsets.add(offset) + } + val blocks = + footer.getBlocks().asScala.flatMap { block: BlockMetaData => + if (offsets.contains(block.getStartingPos())) { + Some(block) + } else None + }.toIndexedSeq + + // verify we found them all + if (blocks.size != rowGroupOffsets.length) { + val foundRowGroupOffsets = new Array[Long](footer.getBlocks().size) + + (0 until foundRowGroupOffsets.length).foreach { i => + foundRowGroupOffsets(i) = footer.getBlocks().get(i).getStartingPos() + } + // this should never happen. + // provide a good error message in case there's a bug + throw new IllegalStateException( + s"""All the offsets listed in the split should be found in the file. + expected: $rowGroupOffsets + found: $blocks + out of: $foundRowGroupOffsets + in range ${split.getStart}, ${split.getEnd}""") + } + (blocks, footer) + } + + if (!blocks.isEmpty) { + checkDeltaByteArrayProblem(footer.getFileMetaData(), configuration, blocks(0)) + } + + val fileSchema: MessageType = footer.getFileMetaData.getSchema + + internalReader = InternalParquetRecordReader( + readSupport, + fileSchema, + footer.getFileMetaData, + path, + blocks, + configuration) + } + + private[this] def checkDeltaByteArrayProblem(meta: FileMetaData, conf: Configuration, block: BlockMetaData) { + // splitting files? + if (conf.getBoolean(ParquetInputFormat.SPLIT_FILES, true)) { + // this is okay if not using DELTA_BYTE_ARRAY with the bug + val encodings: JSet[Encoding] = new HashSet[Encoding]() + block.getColumns.asScala.foreach{ column: ColumnChunkMetaData => + encodings.addAll(column.getEncodings()) + } + encodings.asScala.foreach { encoding => + if (CorruptDeltaByteArrays.requiresSequentialReads(meta.getCreatedBy, encoding)) { + throw new ParquetDecodingException("Cannot read data due to " + + "PARQUET-246: to read safely, set " + SPLIT_FILES + " to false") + } + } + } + } + + /** + * {@inheritDoc} + */ + override def nextKeyValue(): Boolean = internalReader.nextKeyValue() + + private[this] def toParquetSplit(split: InputSplit): ParquetInputSplit = + split match { + case pis: ParquetInputSplit => pis + case fs: FileSplit => ParquetInputSplitBridge.from(fs) + case ofs: org.apache.hadoop.mapred.FileSplit => ParquetInputSplitBridge.from(ofs) + case _ => throw new IllegalArgumentException("Invalid split (not a FileSplit or ParquetInputSplit): " + split) + } +} diff --git a/lui-core/src/main/scala/com/twitter/lui/inputformat/SplitInfo.scala b/lui-core/src/main/scala/com/twitter/lui/inputformat/SplitInfo.scala new file mode 100644 index 0000000000..d23db58984 --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/inputformat/SplitInfo.scala @@ -0,0 +1,97 @@ +package com.twitter.lui.inputformat + +import java.lang.Boolean.TRUE +import org.apache.parquet.Preconditions.checkArgument + +import java.io.IOException +import java.util.ArrayList +import java.util.Arrays +import java.util.Collection +import java.util.Collections +import java.util.Comparator +import java.util.HashMap +import java.util.HashSet +import java.util.{ List => JList, Map => JMap, Set => JSet } + +import org.apache.hadoop.conf.Configurable +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.BlockLocation +import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.PathFilter +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapreduce.InputSplit +import org.apache.hadoop.mapreduce.Job +import org.apache.hadoop.mapreduce.JobContext +import org.apache.hadoop.mapreduce.RecordReader +import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat + +import org.apache.hadoop.mapreduce.lib.input.FileSplit +import org.apache.parquet.Log +import org.apache.parquet.Preconditions +import org.apache.parquet.filter.UnboundRecordFilter +import org.apache.parquet.filter2.compat.FilterCompat +import org.apache.parquet.filter2.compat.FilterCompat.Filter +import org.apache.parquet.filter2.compat.RowGroupFilter +import org.apache.parquet.filter2.predicate.FilterPredicate +import org.apache.parquet.hadoop.api.InitContext +import com.twitter.lui.hadoop.ReadSupport +import com.twitter.lui.hadoop.ReadContext +import org.apache.parquet.hadoop.metadata.BlockMetaData +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData +import org.apache.parquet.hadoop.metadata.ParquetMetadata +import org.apache.parquet.hadoop.util.ConfigurationUtil +import org.apache.parquet.hadoop.util.ContextUtil +import org.apache.parquet.hadoop.util.HiddenFileFilter +import org.apache.parquet.hadoop.util.SerializationUtil +import org.apache.parquet.io.ParquetDecodingException +import org.apache.parquet.schema.MessageType +import org.apache.parquet.schema.MessageTypeParser +import org.apache.parquet.hadoop.ParquetInputSplit +import scala.collection.JavaConverters._ + +private[inputformat] class SplitInfo(hdfsBlock: BlockLocation) { + private[this] val rowGroups = new ArrayList[BlockMetaData]() + var compressedByteSize: Long = 0L + + def addRowGroup(rowGroup: BlockMetaData) { + rowGroups.add(rowGroup) + compressedByteSize += rowGroup.getCompressedSize() + } + + def getCompressedByteSize: Long = compressedByteSize + + def getRowGroups: JList[BlockMetaData] = rowGroups + + def getRowGroupCount: Int = rowGroups.size() + + def getParquetInputSplit(fileStatus: FileStatus, + requestedSchema: String, + readSupportMetadata: Map[String, String]): ParquetInputSplit = { + + val requested: MessageType = MessageTypeParser.parseMessageType(requestedSchema) + + val length: Long = getRowGroups.asScala.flatMap { block: BlockMetaData => + block.getColumns.asScala.map { column => + if (requested.containsPath(column.getPath.toArray())) { + column.getTotalSize + } else 0L + } + }.sum + + val lastRowGroup: BlockMetaData = getRowGroups.get(this.getRowGroupCount - 1) + val end: Long = lastRowGroup.getStartingPos + lastRowGroup.getTotalByteSize + + val rowGroupOffsets = rowGroups.asScala.map(_.getStartingPos).toArray + + new ParquetInputSplit( + fileStatus.getPath, + hdfsBlock.getOffset, + end, + length, + hdfsBlock.getHosts, + rowGroupOffsets) + } +} diff --git a/lui-core/src/main/scala/com/twitter/lui/inputformat/codec/BytesDecompressor.scala b/lui-core/src/main/scala/com/twitter/lui/inputformat/codec/BytesDecompressor.scala new file mode 100644 index 0000000000..180e63f513 --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/inputformat/codec/BytesDecompressor.scala @@ -0,0 +1,56 @@ +/* + * 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 com.twitter.lui.inputformat.codec + +import java.io.ByteArrayInputStream +import java.io.ByteArrayOutputStream +import java.io.IOException +import java.io.InputStream + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.io.compress.CodecPool +import org.apache.hadoop.io.compress.CompressionCodec +import org.apache.hadoop.io.compress.CompressionOutputStream +import org.apache.hadoop.io.compress.Compressor +import org.apache.hadoop.io.compress.Decompressor +import org.apache.hadoop.util.ReflectionUtils + +import org.apache.parquet.bytes.BytesInput +import org.apache.parquet.hadoop.metadata.CompressionCodecName + +class BytesDecompressor(codec: CompressionCodec) { + private[this] val decompressor: Decompressor = if (codec != null) { + CodecPool.getDecompressor(codec) + } else null + + def decompress(bytes: BytesInput, uncompressedSize: Int): BytesInput = + if (codec != null) { + decompressor.reset() + val is: InputStream = codec.createInputStream(new ByteArrayInputStream(bytes.toByteArray()), decompressor) + BytesInput.from(is, uncompressedSize) + } else { + bytes + } + + private[codec] def release() { + if (decompressor != null) { + CodecPool.returnDecompressor(decompressor) + } + } +} diff --git a/lui-core/src/main/scala/com/twitter/lui/inputformat/codec/CodecFactory.scala b/lui-core/src/main/scala/com/twitter/lui/inputformat/codec/CodecFactory.scala new file mode 100644 index 0000000000..02923755f8 --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/inputformat/codec/CodecFactory.scala @@ -0,0 +1,72 @@ +/* + * 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 com.twitter.lui.inputformat.codec + +import java.io.ByteArrayInputStream +import java.io.ByteArrayOutputStream +import java.io.IOException +import java.io.InputStream +import scala.collection.mutable.{ Map => MMap } + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.io.compress.CodecPool +import org.apache.hadoop.io.compress.CompressionCodec +import org.apache.hadoop.io.compress.CompressionOutputStream +import org.apache.hadoop.io.compress.Compressor +import org.apache.hadoop.io.compress.Decompressor +import org.apache.hadoop.util.ReflectionUtils + +import org.apache.parquet.bytes.BytesInput +import org.apache.parquet.hadoop.metadata.CompressionCodecName + +class CodecFactory(configuration: Configuration) { + private[this] val decompressors = MMap[CompressionCodecName, BytesDecompressor]() + private[this] val codecByName = MMap[String, CompressionCodec]() + + /** + * + * @param codecName the requested codec + * @return the corresponding hadoop codec. null if UNCOMPRESSED + */ + private[this] def getCodec(codecName: CompressionCodecName): CompressionCodec = { + val codecClassName: String = codecName.getHadoopCompressionCodecClassName + if (codecClassName == null) + null + else + codecByName.getOrElseUpdate(codecClassName, { + try { + ReflectionUtils.newInstance(Class.forName(codecClassName), configuration).asInstanceOf[CompressionCodec] + } catch { + case e: ClassNotFoundException => throw new Exception("Class " + codecClassName + " was not found", e) + } + }) + } + + def getDecompressor(codecName: CompressionCodecName): BytesDecompressor = + decompressors.getOrElseUpdate(codecName, { + new BytesDecompressor(getCodec(codecName)) + }) + + def release() { + decompressors.values.foreach{ decompressor => + decompressor.release() + } + decompressors.clear() + } +} diff --git a/lui-core/src/main/scala/com/twitter/lui/record_reader/InternalParquetRecordReader.scala b/lui-core/src/main/scala/com/twitter/lui/record_reader/InternalParquetRecordReader.scala new file mode 100644 index 0000000000..acef742628 --- /dev/null +++ b/lui-core/src/main/scala/com/twitter/lui/record_reader/InternalParquetRecordReader.scala @@ -0,0 +1,232 @@ +/* + * 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 com.twitter.lui.record_reader + +import java.io.IOException +import java.util.Collections +import java.util.HashMap +import java.util.HashSet +import java.util.{ List => JList, Map => JMap, Set => JSet } + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.parquet.hadoop.UnmaterializableRecordCounter +import org.apache.parquet.Log +import org.apache.parquet.column.ColumnDescriptor +import org.apache.parquet.column.page.PageReadStore +import org.apache.parquet.hadoop.api.InitContext +import com.twitter.lui.hadoop.{ ReadSupport, ReadContext } +import org.apache.parquet.hadoop.metadata.BlockMetaData +import org.apache.parquet.hadoop.metadata.FileMetaData +import org.apache.parquet.hadoop.util.counters.BenchmarkCounter +import org.apache.parquet.io.ParquetDecodingException +import org.apache.parquet.io.api.RecordMaterializer +import org.apache.parquet.io.api.RecordMaterializer.RecordMaterializationException +import org.apache.parquet.schema.GroupType +import org.apache.parquet.schema.MessageType +import org.apache.parquet.schema.Type +import scala.collection.JavaConverters._ + +import org.apache.parquet.Log.DEBUG +import com.twitter.lui.inputformat.ParquetInputFormat.STRICT_TYPE_CHECKING +import com.twitter.lui.inputformat.ParquetFileReader +import com.twitter.lui.{ RecordReader, MessageColumnIO, MessageColumnIOFactory } + +object InternalParquetRecordReader { + private val LOG: Log = Log.getLog(getClass) + + def apply[T](readSupport: ReadSupport[T], + fileSchema: MessageType, + parquetFileMetadata: FileMetaData, + file: Path, + blocks: IndexedSeq[BlockMetaData], + configuration: Configuration): InternalParquetRecordReader[T] = { + + // initialize a ReadContext for this file + val fileMetadata: JMap[String, String] = parquetFileMetadata.getKeyValueMetaData + + // Read all columns from disk + val columnsToReadFromDisk: List[ColumnDescriptor] = fileSchema.getColumns.asScala.toList + + val reader = new ParquetFileReader(configuration, parquetFileMetadata, file, blocks, columnsToReadFromDisk) + + val total: Long = blocks.map(_.getRowCount).sum + + val unmaterializableRecordCounter = new UnmaterializableRecordCounter(configuration, total) + LOG.info("RecordReader initialized will read a total of " + total + " records.") + + InternalParquetRecordReader[T]( + readSupport, + file, + parquetFileMetadata.getCreatedBy, + fileSchema, + configuration.getBoolean(STRICT_TYPE_CHECKING, true), + reader, + unmaterializableRecordCounter, + total, + configuration, + fileMetadata) + } + + def toSetMultiMap[K, V](map: JMap[K, V]): JMap[K, JSet[V]] = { + val setMultiMap = new HashMap[K, JSet[V]]() + map.entrySet.asScala.foreach { entry => + val set = new HashSet[V]() + set.add(entry.getValue()) + setMultiMap.put(entry.getKey(), Collections.unmodifiableSet(set)) + } + Collections.unmodifiableMap(setMultiMap) + } + +} + +case class InternalParquetRecordReader[T] private (readSupport: ReadSupport[T], + file: Path, + createdBy: String, + fileSchema: MessageType, + strictTypeChecking: Boolean, + reader: ParquetFileReader, + unmaterializableRecordCounter: UnmaterializableRecordCounter, + total: Long, + configuration: Configuration, + fileMetadata: JMap[String, String]) { + import InternalParquetRecordReader._ + + val columnCount: Int = fileSchema.getPaths.size + + val readContext: ReadContext = readSupport.init(new InitContext( + configuration, + toSetMultiMap(fileMetadata), + fileSchema)) + + private[this] var currentValue: T = _ + private[this] var current: Long = 0 + private[this] var currentBlock: Int = -1 + private[this] var recordReader: RecordReader[T] = _ + + private[this] var totalTimeSpentReadingBytes: Long = 0L + private[this] var totalTimeSpentProcessingRecords: Long = 0L + private[this] var startedAssemblingCurrentBlockAt: Long = 0L + + private[this] var totalCountLoadedSoFar: Long = 0L + + private[this] def checkRead() { + if (current == totalCountLoadedSoFar) { + if (current != 0) { + totalTimeSpentProcessingRecords += (System.currentTimeMillis() - startedAssemblingCurrentBlockAt) + if (Log.INFO) { + LOG.info(s"Assembled and processed $totalCountLoadedSoFar records from $columnCount columns in $totalTimeSpentProcessingRecords ms: ${totalCountLoadedSoFar.toFloat / totalTimeSpentProcessingRecords} rec/ms, ${(totalCountLoadedSoFar.toFloat * columnCount / totalTimeSpentProcessingRecords)} cell/ms") + val totalTime: Long = totalTimeSpentProcessingRecords + totalTimeSpentReadingBytes + if (totalTime != 0) { + val percentReading: Long = 100 * totalTimeSpentReadingBytes / totalTime + val percentProcessing: Long = 100 * totalTimeSpentProcessingRecords / totalTime + LOG.info(s"time spent so far $percentReading % reading ($totalTimeSpentReadingBytes ms) and $percentProcessing % processing ($totalTimeSpentProcessingRecords ms)") + } + } + } + + LOG.info("at row " + current + ". reading next block") + val t0: Long = System.currentTimeMillis() + val pages: PageReadStore = reader.readNextRowGroup() + if (pages == null) { + throw new IOException(s"expecting more rows but reached last block. Read $current out of $total") + } + val timeSpentReading: Long = System.currentTimeMillis() - t0 + totalTimeSpentReadingBytes += timeSpentReading + BenchmarkCounter.incrementTime(timeSpentReading) + if (Log.INFO) LOG.info("block read in memory in " + timeSpentReading + " ms. row count = " + pages.getRowCount()) + + recordReader = MessageColumnIOFactory.getRecordReader[T]( + pages, + configuration, + createdBy, + fileMetadata.asScala.toMap, + fileSchema, + readContext, + readSupport, + strictTypeChecking) + + startedAssemblingCurrentBlockAt = System.currentTimeMillis() + totalCountLoadedSoFar += pages.getRowCount() + currentBlock += 1 + } + } + + def close() { + if (reader != null) { + reader.close() + } + } + + def getCurrentValue(): T = currentValue + + def getProgress: Float = (current.toFloat / total) + + private[this] def contains(group: GroupType, path: Array[String], index: Int): Boolean = + if (index == path.length) { + false + } else { + if (group.containsField(path(index))) { + val tpe: Type = group.getType(path(index)) + if (tpe.isPrimitive) { + return index + 1 == path.length + } else { + return contains(tpe.asGroupType, path, index + 1) + } + } + false + } + + def nextKeyValue(): Boolean = { + var recordFound: Boolean = false + + // if there are records left and we haven't found one yet + while (current < total && !recordFound) { + + try { + checkRead() + current += 1 + + currentValue = recordReader.read() // might fail with a RecordMaterializationException + + if (currentValue == null) { + // only happens with FilteredRecordReader at end of block + current = totalCountLoadedSoFar + if (DEBUG) LOG.debug("filtered record reader reached end of block") + } else { + + recordFound = true + } + + if (DEBUG) LOG.debug("read value: " + currentValue) + } catch { + case e: RecordMaterializationException => + // this might throw, but it's fatal if it does. + unmaterializableRecordCounter.incErrors(e) + if (DEBUG) LOG.debug("skipping a corrupt record") + + case e: RuntimeException => + throw new ParquetDecodingException(s"Can not read value at $current in block $currentBlock in file $file", e) + } + } + recordFound + } + +} diff --git a/lui-core/src/main/scala/org/apache/parquet/column/EncodingAccessor.scala b/lui-core/src/main/scala/org/apache/parquet/column/EncodingAccessor.scala new file mode 100644 index 0000000000..c466e1e14d --- /dev/null +++ b/lui-core/src/main/scala/org/apache/parquet/column/EncodingAccessor.scala @@ -0,0 +1,6 @@ +package org.apache.parquet.column + +object EncodingAccessor { + def getMaxLevel(encoding: Encoding, descriptor: ColumnDescriptor, valuesType: ValuesType) = + encoding.getMaxLevel(descriptor, valuesType) +} diff --git a/lui-core/src/main/scala/org/apache/parquet/hadoop/ParquetFileWriterBridge.scala b/lui-core/src/main/scala/org/apache/parquet/hadoop/ParquetFileWriterBridge.scala new file mode 100644 index 0000000000..71df97ea98 --- /dev/null +++ b/lui-core/src/main/scala/org/apache/parquet/hadoop/ParquetFileWriterBridge.scala @@ -0,0 +1,7 @@ +package org.apache.parquet.hadoop + +import java.util.{ List => JList } +object ParquetFileWriterBridge { + def getGlobalMetaData(footers: JList[Footer], strict: Boolean = true) = + ParquetFileWriter.getGlobalMetaData(footers, strict) +} \ No newline at end of file diff --git a/lui-core/src/main/scala/org/apache/parquet/hadoop/ParquetInputSplitBridge.scala b/lui-core/src/main/scala/org/apache/parquet/hadoop/ParquetInputSplitBridge.scala new file mode 100644 index 0000000000..10fed46286 --- /dev/null +++ b/lui-core/src/main/scala/org/apache/parquet/hadoop/ParquetInputSplitBridge.scala @@ -0,0 +1,6 @@ +package org.apache.parquet.hadoop + +object ParquetInputSplitBridge { + def from(split: org.apache.hadoop.mapreduce.lib.input.FileSplit): ParquetInputSplit = ParquetInputSplit.from(split) + def from(split: org.apache.hadoop.mapred.FileSplit): ParquetInputSplit = ParquetInputSplit.from(split) +} \ No newline at end of file diff --git a/lui-scalding-scrooge/src/main/scala/com/twitter/lui/scalding/scrooge/LuiScroogeScheme.scala b/lui-scalding-scrooge/src/main/scala/com/twitter/lui/scalding/scrooge/LuiScroogeScheme.scala new file mode 100644 index 0000000000..5ca5e5af5b --- /dev/null +++ b/lui-scalding-scrooge/src/main/scala/com/twitter/lui/scalding/scrooge/LuiScroogeScheme.scala @@ -0,0 +1,43 @@ +package com.twitter.lui.scalding.scrooge + +import cascading.flow.FlowProcess +import cascading.scheme.{ Scheme, SinkCall, SourceCall } +import cascading.tap.Tap +import cascading.tuple.Tuple +import com.twitter.lui.hadoop.ReadSupport +import com.twitter.lui.inputformat.MapRedParquetInputFormat +import com.twitter.lui.inputformat.ParquetInputFormat +import com.twitter.lui.scrooge.LuiScroogeReadSupport +import com.twitter.scrooge.ThriftStruct +import org.apache.hadoop.mapred.{ JobConf, RecordReader } +import org.apache.parquet.column.ColumnDescriptor +import org.apache.parquet.hadoop.mapred.Container +import org.apache.parquet.schema.PrimitiveType +import scala.reflect.ClassTag + +class LuiScroogeScheme[T >: Null <: ThriftStruct: ClassTag]() extends Scheme[JobConf, RecordReader[Void, Container[T]], Void, Void, Void] { + override def sourceConfInit( + flowProcess: FlowProcess[JobConf], + tap: Tap[JobConf, RecordReader[Void, Container[T]], Void], + config: JobConf): Unit = { + config.setInputFormat(classOf[MapRedParquetInputFormat[T]]) + ParquetInputFormat.setReadSupportClass(config, classOf[LuiScroogeReadSupport[T]], implicitly[ClassTag[T]].runtimeClass.asInstanceOf[Class[T]]) + } + + override def source( + flowProcess: FlowProcess[JobConf], + sourceCall: SourceCall[Void, RecordReader[Void, Container[T]]]): Boolean = { + val container = sourceCall.getInput.createValue() + + val hasNext = sourceCall.getInput.next(null, container) + if (!hasNext) false + else if (container == null) true + else { + sourceCall.getIncomingEntry.setTuple(new Tuple(container.get)) + true + } + } + + override def sink(flowProcess: FlowProcess[JobConf], sinkCall: SinkCall[Void, Void]): Unit = ??? + override def sinkConfInit(flowProcess: FlowProcess[JobConf], tap: Tap[JobConf, RecordReader[Void, Container[T]], Void], config: JobConf): Unit = ??? +} \ No newline at end of file diff --git a/lui-scalding-scrooge/src/main/scala/com/twitter/lui/scalding/scrooge/LuiScroogeSource.scala b/lui-scalding-scrooge/src/main/scala/com/twitter/lui/scalding/scrooge/LuiScroogeSource.scala new file mode 100644 index 0000000000..2714b13878 --- /dev/null +++ b/lui-scalding-scrooge/src/main/scala/com/twitter/lui/scalding/scrooge/LuiScroogeSource.scala @@ -0,0 +1,20 @@ +package com.twitter.lui.scalding.scrooge + +import cascading.scheme.Scheme +import com.twitter.scalding._ +import com.twitter.scrooge.ThriftStruct +import org.apache.hadoop.mapred.{ JobConf, OutputCollector, RecordReader } +import scala.reflect.ClassTag + +trait LuiScroogeSource[T >: Null <: ThriftStruct] extends FileSource with SingleMappable[T] with LocalTapSource { + def ct: ClassTag[T] + + override def hdfsScheme: Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], _, _] = { + val scheme = new LuiScroogeScheme[T]()(ct) + HadoopSchemeInstance(scheme.asInstanceOf[Scheme[_, _, _, _, _]]) + } +} + +case class FPLuiScroogeSource[T >: Null <: ThriftStruct: ClassTag](paths: String*) extends FixedPathSource(paths: _*) with LuiScroogeSource[T] { + override def ct = implicitly[ClassTag[T]] +} \ No newline at end of file diff --git a/lui-scalding-scrooge/src/test/scala/com/twitter/lui/scalding/scrooge/IntegrationTests.scala b/lui-scalding-scrooge/src/test/scala/com/twitter/lui/scalding/scrooge/IntegrationTests.scala new file mode 100644 index 0000000000..dcf3b551c7 --- /dev/null +++ b/lui-scalding-scrooge/src/test/scala/com/twitter/lui/scalding/scrooge/IntegrationTests.scala @@ -0,0 +1,632 @@ +/* +Copyright 2012 Twitter, Inc. + +Licensed 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 com.twitter.lui.scalding.scrooge + +import com.twitter.lui.thrift.thrift_scala.test._ +import com.twitter.scalding._ +import com.twitter.scalding.parquet.scrooge.FixedPathParquetScrooge +import org.scalatest.WordSpec +import scala.util.{ Try, Success, Failure } + +object CompareHelper { + def traversableOnceCompare(a: TraversableOnce[Any], b: TraversableOnce[Any]) = + a.toIterator.zip(b.toIterator).zipWithIndex.foreach { + case ((aE, bE), idx) => + Try(recursiveCompare(aE, bE)) match { + case Success(_) => () + case Failure(e) => throw new Exception(s""" +Comparing index ${idx} type: ${aE.getClass} +${e.getMessage} + +Reference List: ${a.mkString(",")} +OtherList: ${b.mkString(",")} +""") + } + } + + def productCompare(a: Product, b: Product): Unit = + traversableOnceCompare(a.productIterator, b.productIterator) + + def recursiveCompare(unknownA: Any, unknownB: Any): Unit = + (unknownA, unknownB) match { + case (a: Product, b: Product) => Try(productCompare(a, b)) match { + case Success(_) => () + case Failure(e) => throw new Exception(s"Comparing type: ${a.getClass}\n${e.getMessage}") + } + case (a: TraversableOnce[Any], b: TraversableOnce[Any]) => Try(traversableOnceCompare(a, b)) match { + case Success(_) => () + case Failure(e) => throw new Exception(s"Comparing type: ${a.getClass}\n${e.getMessage}") + } + case (a, b) => if (a != b) sys.error(s"Reference $a was not equal to test target $b, of type ${a.getClass}") + } +} + +class IntegrationTests extends WordSpec { + // only set this to false in local development, allows faster iteration speed + // by not rebuilding the sample parquet data on every invokation + val SHOULD_DO_WRITE_STEP = true + + import CompareHelper._ + org.apache.log4j.Logger.getLogger("org.apache.parquet.hadoop.ParquetOutputFormat").setLevel(org.apache.log4j.Level.ERROR) + org.apache.log4j.Logger.getLogger("org.apache.parquet.hadoop.codec.CodecConfig").setLevel(org.apache.log4j.Level.ERROR) + org.apache.log4j.Logger.getLogger("org.apache.parquet.hadoop.ColumnChunkPageWriteStore").setLevel(org.apache.log4j.Level.ERROR) + org.apache.log4j.Logger.getLogger("org.apache.parquet.hadoop.thrift.AbstractThriftWriteSupport").setLevel(org.apache.log4j.Level.ERROR) + org.apache.log4j.Logger.getLogger("org.apache.parquet.hadoop.ParquetFileReader").setLevel(org.apache.log4j.Level.ERROR) + org.apache.log4j.Logger.getLogger("org.apache.parquet.Log").setLevel(org.apache.log4j.Level.ERROR) + + java.util.logging.Logger.getLogger("org.apache.parquet.hadoop.ParquetOutputFormat").setLevel(java.util.logging.Level.SEVERE) + java.util.logging.Logger.getLogger("org.apache.parquet.hadoop.codec.CodecConfig").setLevel(java.util.logging.Level.SEVERE) + java.util.logging.Logger.getLogger("org.apache.parquet.hadoop.ColumnChunkPageWriteStore").setLevel(java.util.logging.Level.SEVERE) + java.util.logging.Logger.getLogger("org.apache.parquet.hadoop.thrift.AbstractThriftWriteSupport").setLevel(java.util.logging.Level.SEVERE) + java.util.logging.Logger.getLogger("org.apache.parquet.hadoop.ParquetFileReader").setLevel(java.util.logging.Level.SEVERE) + java.util.logging.Logger.getLogger("org.apache.parquet.Log").setLevel(java.util.logging.Level.SEVERE) + + implicit class MappableExt[T](m: Mappable[T]) { + def iter: Iterator[T] = { + import com.twitter.scalding._ + implicit val cfg = Config.empty + implicit val mode = Local(false) + m.toIterator + } + } + + "StringAndBinary Source should round trip" should { + val writeTarget = new FixedPathParquetScrooge[StringAndBinary](s"/tmp/scalding_StringAndBinary") + val readTarget = new FPLuiScroogeSource[StringAndBinary](s"/tmp/scalding_StringAndBinary") + + val expected: List[StringAndBinary] = (0 until 100).map { i => + val str = i.toString + val bytes = java.nio.ByteBuffer.wrap(str.getBytes("UTF-8")) + StringAndBinary(str, bytes) + }.toList + + val exec = TypedPipe.from(List(1)).flatMap{ _ => + expected + }.writeExecution(writeTarget) + + if (SHOULD_DO_WRITE_STEP) + exec.waitFor(Config.empty, Hdfs(false, new org.apache.hadoop.conf.Configuration)) + + // Do a modulo so we handle some skipping, touch some fields to make sure we keep all the columns in sync too + val expectedWithFilter = expected.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + val actual = readTarget.iter.map { t => + t.hashCode + t + }.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + + val iter1 = expectedWithFilter.iterator + val iter2 = actual.iterator + while (iter1.hasNext) { + assert(iter1.next == iter2.next) + } + } + + "RequiredPrimitiveFixture Source should round trip" should { + val writeTarget = new FixedPathParquetScrooge[RequiredPrimitiveFixture](s"/tmp/scalding_RequiredPrimitiveFixture") + val readTarget = new FPLuiScroogeSource[RequiredPrimitiveFixture](s"/tmp/scalding_RequiredPrimitiveFixture") + + val expected: List[RequiredPrimitiveFixture] = (0 until 100).map { i => + val boolean = (i % 3 == 0) + val byte = i.toByte + val short = i.toShort + val integer = i + val long = i.toLong + val double = i.toDouble + val str = i.toString + RequiredPrimitiveFixture(boolean, byte, short, integer, long, double, str) + }.toList + + val exec = TypedPipe.from(List(1)).flatMap{ _ => + expected + }.writeExecution(writeTarget) + + if (SHOULD_DO_WRITE_STEP) + exec.waitFor(Config.empty, Hdfs(false, new org.apache.hadoop.conf.Configuration)) + + // Do a modulo so we handle some skipping, touch some fields to make sure we keep all the columns in sync too + val expectedWithFilter = expected.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + val actual = readTarget.iter.map { t => + t.hashCode + t + }.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + + val iter1 = expectedWithFilter.iterator + val iter2 = actual.iterator + while (iter1.hasNext) { + assert(iter1.next == iter2.next) + } + } + + "StructWithOptionalInnerStructAndOptionalBooleanInside should round trip" should { + val writeTarget = new FixedPathParquetScrooge[StructWithOptionalInnerStructAndOptionalBooleanInside](s"/tmp/scalding_StructWithOptionalInnerStructAndBooleanInside") + val readTarget = new FPLuiScroogeSource[StructWithOptionalInnerStructAndOptionalBooleanInside](s"/tmp/scalding_StructWithOptionalInnerStructAndBooleanInside") + + val expected: List[StructWithOptionalInnerStructAndOptionalBooleanInside] = (0 until 100).map { i => + if (i % 2 == 4) + StructWithOptionalInnerStructAndOptionalBooleanInside(None) + else { + if (i % 2 == 0) { + StructWithOptionalInnerStructAndOptionalBooleanInside(Some(MaybeBoolean(i.toLong, None))) + } else { + StructWithOptionalInnerStructAndOptionalBooleanInside(Some(MaybeBoolean(i.toLong, Some(i % 3 == 0)))) + } + + } + }.toList + + val exec = TypedPipe.from(List(1)).flatMap{ _ => + expected + }.writeExecution(writeTarget) + + if (SHOULD_DO_WRITE_STEP) + exec.waitFor(Config.empty, Hdfs(false, new org.apache.hadoop.conf.Configuration)) + + // Do a modulo so we handle some skipping, touch some fields to make sure we keep all the columns in sync too + val expectedWithFilter = expected.zipWithIndex.filter(_._2 % 5 == 0).map(_._1) + val actual = readTarget.iter.map { t => + t.hashCode + t + }.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + assert(expected == actual) + } + } + + "StructWithOptionalInnerStruct should round trip" should { + val writeTarget = new FixedPathParquetScrooge[StructWithOptionalInnerStruct](s"/tmp/scalding_StructWithOptionalInnerStruct") + val readTarget = new FPLuiScroogeSource[StructWithOptionalInnerStruct](s"/tmp/scalding_StructWithOptionalInnerStruct") + + val expected: List[StructWithOptionalInnerStruct] = (0 until 100).map { i => + if (i % 2 == 0) + StructWithOptionalInnerStruct(None) + else { + StructWithOptionalInnerStruct(Some(Phone(i + ".. mobile", "work.." + i))) + } + }.toList + + val exec = TypedPipe.from(List(1)).flatMap{ _ => + expected + }.writeExecution(writeTarget) + + if (SHOULD_DO_WRITE_STEP) + exec.waitFor(Config.empty, Hdfs(false, new org.apache.hadoop.conf.Configuration)) + + // Do a modulo so we handle some skipping, touch some fields to make sure we keep all the columns in sync too + val expectedWithFilter = expected.zipWithIndex.filter(_._2 % 5 == 0).map(_._1) + val actual = readTarget.iter.map { t => + t.hashCode + t + }.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + assert(expected == actual) + } + } + + "PrimitiveMap should round trip" should { + val writeTarget = new FixedPathParquetScrooge[PrimitiveMap](s"/tmp/scalding_PrimitiveMap") + val readTarget = new FPLuiScroogeSource[PrimitiveMap](s"/tmp/scalding_PrimitiveMap") + + val expected: List[PrimitiveMap] = (0 until 100).map { i => + val booleanMap = (0 until i + 2).map { idx => (idx % 3 == 0) -> (idx % 3 == 0) }.toMap + val byteMap = (0 until i + 2).map { idx => idx.toByte -> idx.toByte }.toMap + val shortMap = (0 until i + 2).map { idx => idx.toShort -> idx.toShort }.toMap + val integerMap = (0 until i + 2).map { idx => idx -> idx }.toMap + val longMap = (0 until i + 2).map { idx => idx.toLong -> idx.toLong }.toMap + val doubleMap = (0 until i + 2).map { idx => idx.toDouble -> idx.toDouble }.toMap + val stringMap = (0 until i + 2).map { idx => idx.toString -> idx.toString }.toMap + PrimitiveMap(booleanMap, byteMap, shortMap, integerMap, longMap, doubleMap, stringMap) + }.toList + + val exec = TypedPipe.from(List(1)).flatMap{ _ => + expected + }.writeExecution(writeTarget) + + if (SHOULD_DO_WRITE_STEP) + exec.waitFor(Config.empty, Hdfs(false, new org.apache.hadoop.conf.Configuration)) + + // Do a modulo so we handle some skipping, touch some fields to make sure we keep all the columns in sync too + val expectedWithFilter = expected.zipWithIndex.filter(_._2 % 5 == 0).map(_._1) + val actual = readTarget.iter.map { t => + t.hashCode + t + }.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + recursiveCompare(expected, actual) + } + + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + assert(expected == actual) + } + } + + "PrimitiveList should round trip" should { + val writeTarget = new FixedPathParquetScrooge[PrimitiveList](s"/tmp/scalding_PrimitiveList") + val readTarget = new FPLuiScroogeSource[PrimitiveList](s"/tmp/scalding_PrimitiveList") + + val expected: List[PrimitiveList] = (0 until 100).map { i => + val booleanList = (0 until i + 2).map { idx => (idx % 3 == 0) } + val byteList = (0 until i + 2).map { idx => idx.toByte } + val shortList = (0 until i + 2).map { idx => idx.toShort } + val integerList = (0 until i + 2).map { idx => idx } + val longList = (0 until i + 2).map { idx => idx.toLong } + val doubleList = (0 until i + 2).map { idx => idx.toDouble } + val stringList = (0 until i + 2).map { idx => idx.toString } + PrimitiveList(booleanList, byteList, shortList, integerList, longList, doubleList, stringList) + }.toList + + val exec = TypedPipe.from(List(1)).flatMap{ _ => + expected + }.writeExecution(writeTarget) + + if (SHOULD_DO_WRITE_STEP) + exec.waitFor(Config.empty, Hdfs(false, new org.apache.hadoop.conf.Configuration)) + + // Do a modulo so we handle some skipping, touch some fields to make sure we keep all the columns in sync too + val expectedWithFilter = expected.zipWithIndex.filter(_._2 % 5 == 0).map(_._1) + val actual = readTarget.iter.map { t => + t.hashCode + t + }.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + assert(expected == actual) + } + } + + "PrimitiveSet should round trip" should { + val writeTarget = new FixedPathParquetScrooge[PrimitiveSet](s"/tmp/scalding_PrimitiveSet") + val readTarget = new FPLuiScroogeSource[PrimitiveSet](s"/tmp/scalding_PrimitiveSet") + + val expected: List[PrimitiveSet] = (0 until 100).map { i => + val booleanSet = (0 until i + 2).map { idx => (idx % 3 == 0) }.toSet + val byteSet = (0 until i + 2).map { idx => idx.toByte }.toSet + val shortSet = (0 until i + 2).map { idx => idx.toShort }.toSet + val integerSet = (0 until i + 2).map { idx => idx }.toSet + val longSet = (0 until i + 2).map { idx => idx.toLong }.toSet + val doubleSet = (0 until i + 2).map { idx => idx.toDouble }.toSet + val stringSet = (0 until i + 2).map { idx => idx.toString }.toSet + PrimitiveSet(booleanSet, byteSet, shortSet, integerSet, longSet, doubleSet, stringSet) + }.toList + + val exec = TypedPipe.from(List(1)).flatMap{ _ => + expected + }.writeExecution(writeTarget) + + if (SHOULD_DO_WRITE_STEP) + exec.waitFor(Config.empty, Hdfs(false, new org.apache.hadoop.conf.Configuration)) + + // Do a modulo so we handle some skipping, touch some fields to make sure we keep all the columns in sync too + val expectedWithFilter = expected.zipWithIndex.filter(_._2 % 5 == 0).map(_._1) + val actual = readTarget.iter.map { t => + t.hashCode + t + }.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + assert(expected == actual) + } + } + + // Simple as it can be, has just 2 fields + "Phone Source should round trip" should { + val writeTarget = new FixedPathParquetScrooge[Phone](s"/tmp/scalding_phone") + val readTarget = new FPLuiScroogeSource[Phone](s"/tmp/scalding_phone") + + val expected: List[Phone] = (0 until 100).map { i => + Phone(i + ".. mobile", "work.." + i) + }.toList + + val exec = TypedPipe.from(List(1)).flatMap{ _ => + (0 until 100).map { i => + Phone(i + ".. mobile", "work.." + i) + }.toList + }.writeExecution(writeTarget) + + if (SHOULD_DO_WRITE_STEP) + exec.waitFor(Config.empty, Hdfs(false, new org.apache.hadoop.conf.Configuration)) + + // Do a modulo so we handle some skipping, touch some fields to make sure we keep all the columns in sync too + val expectedWithFilter = expected.zipWithIndex.filter(_._2 % 5 == 0).map(_._1) + val actual = readTarget.iter.map { t => + t.hashCode + t + }.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + assert(expected == actual) + } + } + + // This handles nested structs + "AStructWithAStruct Source should round trip" should { + val writeTarget = new FixedPathParquetScrooge[AStructWithAStruct](s"/tmp/scalding_AStructWithAStruct") + val readTarget = new FPLuiScroogeSource[AStructWithAStruct](s"/tmp/scalding_AStructWithAStruct") + + val expected = (0 until 100).map { i => + AStructWithAStruct(AString(i.toString)) + }.toList + + val exec = TypedPipe.from(List(1)).flatMap{ _ => + (0 until 100).map { i => + AStructWithAStruct(AString(i.toString)) + }.toList + + }.writeExecution(writeTarget) + + if (SHOULD_DO_WRITE_STEP) + exec.waitFor(Config.empty, Hdfs(false, new org.apache.hadoop.conf.Configuration)) + + // Do a modulo so we handle some skipping, touch some fields to make sure we keep all the columns in sync too + val expectedWithFilter = expected.zipWithIndex.filter(_._2 % 5 == 0).map(_._1) + val actual = readTarget.iter.map { t => + t.hashCode + t + }.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + assert(expected == actual) + } + } + + // This handles nested structs + "RequiredListFixture Source should round trip" should { + val writeTarget = new FixedPathParquetScrooge[RequiredListFixture](s"/tmp/scalding_RequiredListFixture") + val readTarget = new FPLuiScroogeSource[RequiredListFixture](s"/tmp/scalding_RequiredListFixture") + + val expected = (0 until 100).map { i => + val info = if (i % 2 == 0) Some(i.toString + "info") else None + val lst = (0 until i + 2).map { idx => Name("Major loop level: " + i.toString + ",minor:" + idx, Some("Major loop level: " + i.toString + "\t" + idx.toString)) } + RequiredListFixture(info, List(NameList(lst))) + }.toIterator + + val exec = TypedPipe.from(List(1)).flatMap{ _ => + (0 until 100).map { i => + val info = if (i % 2 == 0) Some(i.toString + "info") else None + val lst = (0 until i + 2).map { idx => Name("Major loop level: " + i.toString + ",minor:" + idx, Some("Major loop level: " + i.toString + "\t" + idx.toString)) } + RequiredListFixture(info, List(NameList(lst))) + }.toList + + }.writeExecution(writeTarget) + + if (SHOULD_DO_WRITE_STEP) + exec.waitFor(Config.empty, Hdfs(false, new org.apache.hadoop.conf.Configuration)) + + // Do a modulo so we handle some skipping, touch some fields to make sure we keep all the columns in sync too + val expectedWithFilter = expected.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + val actual = readTarget.iter.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + assert(expected == actual) + } + } + + // This RequiredMapFixture nested structs + "RequiredMapFixture Source should round trip" should { + val writeTarget = new FixedPathParquetScrooge[RequiredMapFixture](s"/tmp/scalding_RequiredMapFixture") + val readTarget = new FPLuiScroogeSource[RequiredMapFixture](s"/tmp/scalding_RequiredMapFixture") + + val expected = (0 until 100).map { i => + val info = if (i % 2 == 0) Some(i.toString + "info") else None + val mapD = (0 until i + 2).map{ idx => + s"${i}_$idx" -> s"${i}_$idx" + }.toMap + RequiredMapFixture(info, mapD) + }.toIterator + + val exec = TypedPipe.from(List(1)).flatMap{ _ => + expected + }.writeExecution(writeTarget) + + if (SHOULD_DO_WRITE_STEP) + exec.waitFor(Config.empty, Hdfs(false, new org.apache.hadoop.conf.Configuration)) + + // Do a modulo so we handle some skipping, touch some fields to make sure we keep all the columns in sync too + val expectedWithFilter = expected.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + val actual = readTarget.iter.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + assert(expected == actual) + } + } + + // This RequiredMapFixture nested structs + "RequiredEnumMapFixture Source should round trip" should { + val writeTarget = new FixedPathParquetScrooge[RequiredEnumMapFixture](s"/tmp/scalding_RequiredEnumMapFixture") + val readTarget = new FPLuiScroogeSource[RequiredEnumMapFixture](s"/tmp/scalding_RequiredEnumMapFixture") + + val expected = (0 until 100).map { i => + val info = if (i % 2 == 0) Some(i.toString + "info") else None + val mapD: Map[Operation, String] = (0 until i + 2).map{ idx => + val requiredEnum = (idx % 4) match { + case 0 => Operation.Add + case 1 => Operation.Subtract + case 2 => Operation.Multiply + case 3 => Operation.Divide + } + requiredEnum -> s"${i}_$idx" + }.toMap + RequiredEnumMapFixture(info, mapD) + }.toIterator + + val exec = TypedPipe.from(List(1)).flatMap{ _ => + expected + }.writeExecution(writeTarget) + + if (SHOULD_DO_WRITE_STEP) + exec.waitFor(Config.empty, Hdfs(false, new org.apache.hadoop.conf.Configuration)) + + // Do a modulo so we handle some skipping, touch some fields to make sure we keep all the columns in sync too + val expectedWithFilter = expected.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + val actual = readTarget.iter.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + assert(expected == actual) + } + } + + // This handles nested structs + "TestFieldOfEnum Source should round trip" should { + val writeTarget = new FixedPathParquetScrooge[TestFieldOfEnum](s"/tmp/scalding_TestFieldOfEnum") + val readTarget = new FPLuiScroogeSource[TestFieldOfEnum](s"/tmp/scalding_TestFieldOfEnum") + + val expected: List[TestFieldOfEnum] = (0 until 100).map { i => + val requiredEnum = (i % 4) match { + case 0 => Operation.Add + case 1 => Operation.Subtract + case 2 => Operation.Multiply + case 3 => Operation.Divide + } + + val optionalEnum = ((i + 1) % 4) match { + case 0 => Operation.Add + case 1 => Operation.Subtract + case 2 => Operation.Multiply + case 3 => Operation.Divide + } + TestFieldOfEnum(requiredEnum, Some(optionalEnum)) + + }.toList + + val exec = TypedPipe.from(List(1)).flatMap{ _ => + expected + }.writeExecution(writeTarget) + + if (SHOULD_DO_WRITE_STEP) + exec.waitFor(Config.empty, Hdfs(false, new org.apache.hadoop.conf.Configuration)) + + // Do a modulo so we handle some skipping, touch some fields to make sure we keep all the columns in sync too + val expectedWithFilter = expected.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + val actual = readTarget.iter.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toList + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + assert(expected == actual) + } + } + + "TestCaseNestedLists Source should round trip" should { + val writeTarget = new FixedPathParquetScrooge[TestCaseNestedLists](s"/tmp/scalding_TestCaseNestedLists") + val readTarget = new FPLuiScroogeSource[TestCaseNestedLists](s"/tmp/scalding_TestCaseNestedLists") + + val expected: List[TestCaseNestedLists] = (0 until 50).map { i => + val evtList = (0 until i).map { idx => + val entries = (0 until idx).map { ydx => + val primitiveList = (0 until ydx).map { jdx => + jdx.toLong + } + if (ydx % 3 == 0) EntryAB(idx, None) else EntryAB(idx, Some(primitiveList)) + } + if (idx % 2 == 0) EvtDetails(entries) else EvtDetails(null) + } + if (i % 3 == 0) TestCaseNestedLists(Some(evtList)) else TestCaseNestedLists(None) + }.toList + + val exec = TypedPipe.from(List(1)).flatMap{ _ => + expected + }.writeExecution(writeTarget) + + if (SHOULD_DO_WRITE_STEP) + exec.waitFor(Config.empty, Hdfs(false, new org.apache.hadoop.conf.Configuration)) + + // Do a modulo so we handle some skipping, touch some fields to make sure we keep all the columns in sync too + val expectedWithFilter = expected.zipWithIndex.toList //.filter(_._2 % 5 == 0).map(_._1).toList + val actual = readTarget.iter.zipWithIndex.toList //filter(_._2 % 5 == 0).map(_._1).toList + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + recursiveCompare(expected, actual) + } + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + assert(expected == actual) + } + } + + "TestUnion Source should round trip" should { + val writeTarget = new FixedPathParquetScrooge[TestUnion](s"/tmp/scalding_TestUnion") + val readTarget = new FPLuiScroogeSource[TestUnion](s"/tmp/scalding_TestUnion") + + val expected: List[TestUnion] = (0 until 50).map { i => + val person = TestPerson(Name(i.toString), Some(i), null, null) + val mapComplex = TestMapComplex(Map()) + if (i % 2 == 0) { + TestUnion.FirstPerson(person) + } else TestUnion.SecondMap(mapComplex) + }.toList + + val exec = TypedPipe.from(List(1)).flatMap{ _ => + expected + }.writeExecution(writeTarget) + + if (SHOULD_DO_WRITE_STEP) + exec.waitFor(Config.empty, Hdfs(false, new org.apache.hadoop.conf.Configuration)) + + // Do a modulo so we handle some skipping, touch some fields to make sure we keep all the columns in sync too + val expectedWithFilter = expected.toIterator.map(_.hashCode).toList + val actual = readTarget.iter.map(_.hashCode).toList + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + recursiveCompare(expected, actual) + } + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + assert(expected == actual) + } + } + + "SimpleListList Source should round trip" should { + val writeTarget = new FixedPathParquetScrooge[SimpleListList](s"/tmp/scalding_SimpleListList") + val readTarget = new FPLuiScroogeSource[SimpleListList](s"/tmp/scalding_SimpleListList") + + val expected: List[SimpleListList] = (0 until 50).map { i => + SimpleListList((0 until i + 2).map { j => + (0 until j + 2).map { k => + s"$i $j $k" + } + }) + }.toList + + val exec = TypedPipe.from(List(1)).flatMap{ _ => + expected + }.writeExecution(writeTarget) + + if (SHOULD_DO_WRITE_STEP) + exec.waitFor(Config.empty, Hdfs(false, new org.apache.hadoop.conf.Configuration)) + + // Do a modulo so we handle some skipping, touch some fields to make sure we keep all the columns in sync too + val expectedWithFilter = expected.toIterator.map(_.hashCode).toList + val actual = readTarget.iter.map(_.hashCode).toList + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + recursiveCompare(expected, actual) + } + expectedWithFilter.zip(actual).foreach { + case (expected, actual) => + assert(expected == actual) + } + } + +} + diff --git a/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/LuiScroogeReadSupport.scala b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/LuiScroogeReadSupport.scala new file mode 100644 index 0000000000..7ea455320e --- /dev/null +++ b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/LuiScroogeReadSupport.scala @@ -0,0 +1,198 @@ +package com.twitter.lui.scrooge + +import com.twitter.lui.column_reader._ +import com.twitter.lui.hadoop.ReadContext +import com.twitter.lui.hadoop.ReadSupport +import com.twitter.lui.MessageColumnIOFactory +import com.twitter.lui.{ MessageColumnIO, GroupColumnIO, PrimitiveColumnIO, ColumnIO } +import com.twitter.lui.{ RecordReader => ParquetRecordReader } +import com.twitter.algebird.Monoid + +import com.twitter.scrooge.ThriftStruct +import com.twitter.scrooge.{ ThriftStructCodec3, ThriftStructMetaData, ThriftStructFieldInfo } + +import org.apache.hadoop.conf.Configuration + +import org.apache.parquet.column.ColumnDescriptor +import org.apache.parquet.column.page.PageReadStore +import org.apache.parquet.hadoop.api.InitContext +import org.apache.parquet.hadoop.mapred.Container +import org.apache.parquet.schema.PrimitiveType +import org.apache.parquet.schema.{ MessageType, Type => ParquetSchemaType } +import org.apache.parquet.thrift.struct.ThriftType +import org.apache.parquet.thrift.ThriftMetaData +import org.apache.parquet.VersionParser.ParsedVersion + +import scala.collection.JavaConverters._ +import scala.collection.mutable.{ Map => MMap } +import scala.reflect.ClassTag + +case class MaxLvls(maxDefLvl: Int, maxRepLvl: Int) + +object LuiScroogeReadSupport { + private[this] val cache = MMap[(String, Int), ScroogeGenerator]() + + def getGenerator(className: String, thriftColumns: Map[String, (ParquetFieldInfo, Option[Int], Option[ParquetThriftEnumInfo], Vector[Short])]): ScroogeGenerator = + cache.synchronized { + val infoIdentifier = thriftColumns.toList.map(_.hashCode).sorted.hashCode + cache.getOrElseUpdate((className, infoIdentifier), ScroogeGenerator(thriftColumns)) + } + + def getVersion(createdBy: String): ParsedVersion = { + import org.apache.parquet.VersionParser + scala.util.Try(VersionParser.parse(createdBy)).getOrElse(null) + } + +} + +class LuiScroogeReadSupport[T >: Null <: ThriftStruct] extends ReadSupport[T] { + import LuiScroogeReadSupport._ + + private[this] def buildMapping(msg: MessageColumnIO): Map[List[String], MaxLvls] = { + @annotation.tailrec + def go(toVisit: Seq[ColumnIO], acc: Map[List[String], MaxLvls]): Map[List[String], MaxLvls] = { + toVisit match { + case GroupColumnIO(_, _, _, maxRep, maxDef, fPath, _, _, children) :: tail => + val newMap = acc + (fPath.toList -> MaxLvls(maxDef, maxRep)) + go(children.toList ::: tail, newMap) + case PrimitiveColumnIO(_, _, _, _, maxRep, maxDef, fPath, _, _) :: tail => + val newMap = acc + (fPath.toList -> MaxLvls(maxDef, maxRep)) + go(tail, newMap) + case Nil => acc + } + } + go(List(msg.root), Map()) + } + + override def getRecordReader( + columns: PageReadStore, + configuration: Configuration, + createdBy: String, + fileMetadata: Map[String, String], + fileSchema: MessageType, + readContext: ReadContext, + strictTypeChecking: Boolean): ParquetRecordReader[T] = { + + val messageColumnIO = MessageColumnIOFactory.buildMessageColumnIO( + readContext, + fileSchema, + createdBy, + strictTypeChecking) + + val mapping = buildMapping(messageColumnIO) + + val thriftMetaData: ThriftMetaData = ThriftMetaData.fromExtraMetaData(fileMetadata.asJava) + val thriftIdPathToParquetIndex: ParquetMappingToThriftMapping = ThriftIdPathToParquetIndex.get(fileSchema, thriftMetaData.getDescriptor) + require(thriftIdPathToParquetIndex.m.map(_._1).size == thriftIdPathToParquetIndex.m.map(_._1).distinct.size, "Thrift id's should be distinct!") + + val parquetFileThriftEnumMetadata: Map[ThriftIdPath, ParquetThriftEnumInfo] = + ThriftEnumMetadata.get(thriftMetaData.getDescriptor).map { case (k, v) => k -> ParquetThriftEnumInfo(v) } + + val ct = classTag + + val className = ct.runtimeClass.getName + + val writerVersion = getVersion(createdBy) + + val thriftIntermediateData: Seq[(ThriftIdPath, ParquetFieldInfo)] = + thriftIdPathToParquetIndex.m.toSeq.map { + case (tidPath: ThriftIdPath, pfi: ParquetFieldInfo) => + val path = pfi.strPath + + val r = mapping(path.toList) + + val enrichedPfi = pfi.copy(maxDefinitionLevel = r.maxDefLvl, maxRepetitionLevel = r.maxRepLvl) + + if (pfi.parquetSchema.isPrimitive) { + + (tidPath, enrichedPfi) + } else (tidPath, enrichedPfi) + } + + // Here we are aiming to capture columns used for zero detection or are inside collections + // those we just buffer up. + // Otherwise we can go with a simpler mechanism + val reUsedLut = thriftIntermediateData.flatMap { + case (tidPath, pfi) => + val multipleHits = pfi.closestPrimitiveChild match { + case Some(child) if (child != tidPath) => List(tidPath -> 1, child -> 1) + case _ => List(tidPath -> 1) + } + val collectionBased = if (pfi.insideCollection) List(tidPath -> 2) else List() + multipleHits ++ collectionBased + }.groupBy(_._1).map { + case (path, listM) => + path -> listM.map(_._2).sum + } + + val primitiveSet: List[((ThriftIdPath, BaseColumnReader), Int)] = thriftIntermediateData + .toIterator + .filter{ case (_, pfi) => pfi.parquetSchema.isPrimitive } + .map { + case (tidPath, pfi) => + val path = pfi.strPath + val r = mapping(path.toList) + + val pType = pfi.parquetSchema.asPrimitiveType + val columnDescriptor = new ColumnDescriptor( + path.toArray, + pType.getPrimitiveTypeName, + pType.getTypeLength, + r.maxRepLvl, + r.maxDefLvl) + + val pageReader = columns.getPageReader(columnDescriptor) + val col = (pfi.thriftType, reUsedLut(tidPath) > 1) match { + + case (_: ThriftType.StringType, true) => cache.StringColumnReader(writerVersion, columnDescriptor, pageReader, r.maxDefLvl, r.maxRepLvl) + case (_: ThriftType.BoolType, true) => cache.BooleanColumnReader(writerVersion, columnDescriptor, pageReader, r.maxDefLvl, r.maxRepLvl) + case (_: ThriftType.ByteType, true) => cache.IntColumnReader(writerVersion, columnDescriptor, pageReader, r.maxDefLvl, r.maxRepLvl) + case (_: ThriftType.I32Type, true) => cache.IntColumnReader(writerVersion, columnDescriptor, pageReader, r.maxDefLvl, r.maxRepLvl) + case (_: ThriftType.I16Type, true) => cache.IntColumnReader(writerVersion, columnDescriptor, pageReader, r.maxDefLvl, r.maxRepLvl) + case (_: ThriftType.I64Type, true) => cache.LongColumnReader(writerVersion, columnDescriptor, pageReader, r.maxDefLvl, r.maxRepLvl) + case (_: ThriftType.DoubleType, true) => cache.DoubleColumnReader(writerVersion, columnDescriptor, pageReader, r.maxDefLvl, r.maxRepLvl) + case (_: ThriftType.EnumType, true) => cache.StringColumnReader(writerVersion, columnDescriptor, pageReader, r.maxDefLvl, r.maxRepLvl) + + case (_: ThriftType.StringType, false) => noncache.StringColumnReader(writerVersion, columnDescriptor, pageReader, r.maxDefLvl, r.maxRepLvl) + case (_: ThriftType.BoolType, false) => noncache.BooleanColumnReader(writerVersion, columnDescriptor, pageReader, r.maxDefLvl, r.maxRepLvl) + case (_: ThriftType.ByteType, false) => noncache.IntColumnReader(writerVersion, columnDescriptor, pageReader, r.maxDefLvl, r.maxRepLvl) + case (_: ThriftType.I32Type, false) => noncache.IntColumnReader(writerVersion, columnDescriptor, pageReader, r.maxDefLvl, r.maxRepLvl) + case (_: ThriftType.I16Type, false) => noncache.IntColumnReader(writerVersion, columnDescriptor, pageReader, r.maxDefLvl, r.maxRepLvl) + case (_: ThriftType.I64Type, false) => noncache.LongColumnReader(writerVersion, columnDescriptor, pageReader, r.maxDefLvl, r.maxRepLvl) + case (_: ThriftType.DoubleType, false) => noncache.DoubleColumnReader(writerVersion, columnDescriptor, pageReader, r.maxDefLvl, r.maxRepLvl) + case (_: ThriftType.EnumType, false) => noncache.StringColumnReader(writerVersion, columnDescriptor, pageReader, r.maxDefLvl, r.maxRepLvl) + + case _ => sys.error(s"Missing column reader for : ${pfi.thriftType.getClass} -> ${pfi.thriftType}") + } + (tidPath, col) + } + .zipWithIndex + .toList + + val allColData: Array[BaseColumnReader] = primitiveSet.map(_._1._2).toArray + val primitiveLut: Map[ThriftIdPath, Int] = primitiveSet.map{ case ((path, _), idx) => path -> idx }.toMap + + val thriftColumns: Map[String, (ParquetFieldInfo, Option[Int], Option[ParquetThriftEnumInfo], Vector[Short])] = + thriftIntermediateData.map { + case (tidPath, pfi) => + tidPath.toVector.mkString("/") -> (pfi, primitiveLut.get(tidPath), parquetFileThriftEnumMetadata.get(tidPath), tidPath.toVector) + }.toMap + + val scroogeGenerator: ScroogeGenerator = getGenerator(className, thriftColumns) + + val globalMaxRepetitionLevel: Int = allColData.map(_.maxRepetitionLevel).max + + new ParquetRecordReader[T] { + var recordIdx = 0L + val emptyArr = new Array[Int](globalMaxRepetitionLevel + 1) // +1 since we normally just index max repetition levels into this + val builder = scroogeGenerator.buildBuilder[T](className, "", allColData, emptyArr) + def read: T = { + val t = builder(recordIdx) + recordIdx += 1 + t + } + } + } + + override def init(context: InitContext): ReadContext = ReadContext(context.getFileSchema) // no projection yet +} diff --git a/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGenerator.scala b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGenerator.scala new file mode 100644 index 0000000000..1a7dec418a --- /dev/null +++ b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGenerator.scala @@ -0,0 +1,189 @@ +package com.twitter.lui.scrooge + +import com.twitter.algebird.Monoid +import com.twitter.lui.column_reader.BaseColumnReader +import com.twitter.lui.column_reader.cache.CacheColumnReader +import com.twitter.lui.column_reader.noncache.NonCacheColumnReader +import com.twitter.scrooge.ThriftEnum +import com.twitter.scrooge.{ ThriftStruct, ThriftUnion, ThriftStructField } +import com.twitter.scrooge.{ ThriftStructCodec3, ThriftStructFieldInfo, ThriftUnionFieldInfo } +import org.apache.parquet.column.page.PageReadStore +import org.apache.parquet.schema.{ MessageType, Type => ParquetSchemaType } +import org.apache.parquet.thrift.ThriftMetaData +import org.apache.thrift.protocol.TType +import scala.collection.JavaConverters._ +import scala.collection.mutable.{ Map => MMap } +import scala.reflect.ClassTag + +case class ExistingPath(toStr: String) extends AnyVal + +case class CustomCompiler() { + import scala.reflect.runtime.universe._ + import scala.tools.reflect.ToolBox + private[this] val toolbox = runtimeMirror(getClass.getClassLoader).mkToolBox() + + def compile[T](code: String): T = { + val startTime = System.nanoTime() + val compiledCode = toolbox.compile(toolbox.parse(code)) + + val endTime = System.nanoTime() + def timeMs: Double = (endTime - startTime).toDouble / 1000000 + println(s"Compiled Java code (${code.size} bytes) in $timeMs ms") + compiledCode().asInstanceOf[T] + } + +} + +object ScroogeGenerator { + sealed trait GenTypes + case object StructT extends GenTypes + case object PrimitiveT extends GenTypes + case object ListT extends GenTypes + case object SetT extends GenTypes + case object MapT extends GenTypes + + def byteToJavaPrimitive(f: ThriftStructFieldInfo, allowOption: Boolean = true): String = { + if (f.isOptional && allowOption) s"Option[${byteToJavaPrimitive(f, false)}]" else + f.tpe match { + case TType.BOOL => "Boolean" + case TType.BYTE => "Byte" + case TType.DOUBLE => "Double" + case TType.I16 => "Short" + case TType.I32 => "Int" + case TType.I64 => "Long" + case TType.STRING => f.manifest.runtimeClass.getName + case TType.STRUCT => f.manifest.runtimeClass.getName + case TType.MAP => f.manifest.runtimeClass.getName + case TType.LIST => f.manifest.runtimeClass.getName + case TType.SET => f.manifest.runtimeClass.getName + case TType.ENUM => f.manifest.runtimeClass.getName + } + } + + def byteToJavaGetter(f: ThriftStructFieldInfo): String = + f.tpe match { + case 2 => "rdr.getBoolean()" + case 3 => "rdr.getInteger().toByte" + case 4 => "rdr.getDouble()" + case 6 => "rdr.getInteger().toShort" + case 8 => "rdr.getInteger()" + case 10 => "rdr.getLong()" + case 11 if (f.manifest.runtimeClass.getName == "java.nio.ByteBuffer") => "rdr.getBinary()" + case 11 => "rdr.getString()" + case 16 => s"${f.name}enumDecode(rdr.getString())" + case _ => """sys.error("unreachable")""" + } + + def bytePrimtiveZero(f: ThriftStructFieldInfo): String = { + if (f.isOptional) s"None" else + f.tpe match { + case TType.BOOL => "false" + case TType.BYTE => "0" + case TType.DOUBLE => "0.0" + case TType.I16 => "0" + case TType.I32 => "0" + case TType.I64 => "0L" + case TType.STRING => "null" + case TType.ENUM => "null" + case _ => """sys.error("unreachable")""" + } + } + + implicit class ThriftStructFieldInfoExtensions(val tsfie: ThriftStructFieldInfo) extends AnyVal { + def tpe: Byte = tsfie.tfield.`type` + def id: Short = tsfie.tfield.id + + def isStruct: Boolean = tpe == 12 + def toGenT = tpe match { + case 2 => PrimitiveT + case 3 => PrimitiveT + case 4 => PrimitiveT + case 6 => PrimitiveT + case 8 => PrimitiveT + case 10 => PrimitiveT + case TType.STRING => PrimitiveT + case TType.STRUCT => StructT + case TType.LIST => ListT + case TType.SET => SetT + case TType.MAP => MapT + case TType.ENUM => PrimitiveT + } + + def name: String = { + val str = tsfie.tfield.name + str.takeWhile(_ == '_') + str. + split('_'). + filterNot(_.isEmpty). + zipWithIndex.map { + case (part, ind) => + val first = if (ind == 0) part.charAt(0).toLower else part.charAt(0).toUpper + val isAllUpperCase = part.forall(_.isUpper) + val rest = if (isAllUpperCase) part.drop(1).toLowerCase else part.drop(1) + new StringBuilder(part.length).append(first).append(rest) + }.mkString + } + } +} + +case class ScroogeGenerator(mapData: Map[String, (ParquetFieldInfo, Option[Int], Option[ParquetThriftEnumInfo], Vector[Short])]) { + import ScroogeGenerator._ + private[this] val cache = MMap[String, (Array[BaseColumnReader], ScroogeGenerator, Long, Array[Int]) => Any]() + // cache ++= Builders.prebuiltBuilders + private[this] implicit val compiler = CustomCompiler() + + def buildBuilder[T <: ThriftStruct](className: String, + existingPath: String, + data: Array[BaseColumnReader], + repetitionOffsets: Array[Int]): Long => T = { + val mainClass = Class.forName(className).asInstanceOf[Class[T]] + val companionClass = Class.forName(className + "$") // Get the module + + if (classOf[ThriftUnion].isAssignableFrom(mainClass)) { + val fieldInfosMethod = companionClass.getMethod("fieldInfos") + val companionInstance = companionClass.getField("MODULE$").get(null) + val metadata: List[ThriftUnionFieldInfo[_, _]] = fieldInfosMethod.invoke(companionInstance).asInstanceOf[List[ThriftUnionFieldInfo[_, _]]] + buildUnion[T](mainClass, + metadata, + existingPath, + data, + _, + repetitionOffsets) + } else { // ThriftStruct + val fieldInfosMethod = companionClass.getMethod("fieldInfos") + val companionInstance = companionClass.getField("MODULE$").get(null) + val metadata: List[ThriftStructFieldInfo] = fieldInfosMethod.invoke(companionInstance).asInstanceOf[List[ThriftStructFieldInfo]] + buildStruct[T](mainClass, + metadata, + existingPath, + data, + _, + repetitionOffsets) + } + + } + + def buildUnion[T <: ThriftStruct](mainClass: Class[T], + metadata: List[ThriftUnionFieldInfo[_, _]], + existingPath: String, + data: Array[BaseColumnReader], + recordIdx: java.lang.Long, + repetitionOffsets: Array[Int]): T = { + + val builder = cache.getOrElseUpdate(existingPath, + ScroogeGeneratorUnionBuilder[T](mapData, mainClass, metadata, ExistingPath(existingPath), data)).asInstanceOf[((Array[BaseColumnReader], ScroogeGenerator, Long, Array[Int]) => T)] + builder(data, this, recordIdx, repetitionOffsets) + + } + + def buildStruct[T <: ThriftStruct](mainClass: Class[T], + metadata: List[ThriftStructFieldInfo], + existingPath: String, + data: Array[BaseColumnReader], + recordIdx: java.lang.Long, + repetitionOffsets: Array[Int]): T = { + val builder = cache.getOrElseUpdate(existingPath, ScroogeGeneratorStructBuilder[T](mapData, mainClass, metadata, ExistingPath(existingPath), data)).asInstanceOf[((Array[BaseColumnReader], ScroogeGenerator, Long, Array[Int]) => T)] + builder(data, this, recordIdx, repetitionOffsets) + } + +} + diff --git a/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorEnumHelper.scala b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorEnumHelper.scala new file mode 100644 index 0000000000..57bc979e05 --- /dev/null +++ b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorEnumHelper.scala @@ -0,0 +1,82 @@ +package com.twitter.lui.scrooge + +import com.twitter.algebird.Monoid +import com.twitter.lui.column_reader.BaseColumnReader +import com.twitter.lui.column_reader.cache.CacheColumnReader +import com.twitter.lui.column_reader.noncache.NonCacheColumnReader +import com.twitter.scrooge.ThriftEnum +import com.twitter.scrooge.{ ThriftStruct, ThriftUnion, ThriftStructField } +import com.twitter.scrooge.{ ThriftStructCodec3, ThriftStructFieldInfo } +import org.apache.parquet.column.page.PageReadStore +import org.apache.parquet.schema.{ MessageType, Type => ParquetSchemaType } +import org.apache.parquet.thrift.ThriftMetaData +import org.apache.thrift.protocol.TType +import scala.collection.JavaConverters._ +import scala.collection.mutable.{ Map => MMap } +import scala.reflect.ClassTag + +object ScroogeGeneratorEnumHelper { + import ScroogeGenerator._ + + def apply( + metadata: List[ThriftStructFieldInfo], + existingPath: ExistingPath, + cols: Array[BaseColumnReader], + mapData: Map[String, (ParquetFieldInfo, Option[Int], Option[ParquetThriftEnumInfo], Vector[Short])]): String = { + + val enumFunctions = metadata.flatMap { + f: ThriftStructFieldInfo => + val tpeStrNoOption = byteToJavaPrimitive(f, false) + val path = if (existingPath.toStr.isEmpty) f.id.toString else s"${existingPath.toStr}/${f.id}" + mapData.get(path).flatMap { + case (_, _, enumInfoOpti, _) => + enumInfoOpti.map { enumData: ParquetThriftEnumInfo => + + val mapStr = enumData.m.map { + case (strV: String, thriftId: Int) => + s""" "$strV" -> $thriftId """ + }.mkString(",") + raw""" + private[this] val ${f.name}EnumDataMap = Map($mapStr) + @inline + private def ${f.name}enumDecode(strV: String): $tpeStrNoOption = + $tpeStrNoOption.getOrUnknown(${f.name}EnumDataMap(strV)) + """ + } + } + + } + + val enumFunctionsFromMaps = metadata.zipWithIndex.flatMap { + case (f: ThriftStructFieldInfo, fIndx) => + f.toGenT match { + case MapT => + val (keyManifest, valueManifest) = (f.manifest.typeArguments(0), f.manifest.typeArguments(1)) + val (keyClazz, valueClazz) = (keyManifest.runtimeClass, valueManifest.runtimeClass) + List(keyClazz, valueClazz).zipWithIndex.filter(tup => classOf[ThriftEnum].isAssignableFrom(tup._1)).map { + case (clazz, idx) => + val path = if (existingPath.toStr.isEmpty) s"${f.id.toString}/${idx + 1}" else s"${existingPath.toStr}/${f.id}/${idx + 1}" + val (_, _, enumInfoOpti, _) = mapData.get(path).getOrElse { + mapData.keys.foreach(println) + sys.error(s"Could not find $path in map") + } + val enumData = enumInfoOpti.get // must be present or error + val mapStr = enumData.m.map { + case (strV: String, thriftId: Int) => + s""" "$strV" -> $thriftId """ + }.mkString(",") + + s""" + |val enumDataMap${fIndx}_${idx} = Map($mapStr) + |@inline + |private def enumDecode${fIndx}_${idx}(strV: String): ${clazz.getName} = + | ${clazz.getName}.getOrUnknown(enumDataMap${fIndx}_${idx}(strV)) + |""".stripMargin('|') + } + case _ => Nil + } + } + + (enumFunctionsFromMaps ++ enumFunctions).mkString("\n") + } +} diff --git a/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorListSetHelper.scala b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorListSetHelper.scala new file mode 100644 index 0000000000..6783e9582c --- /dev/null +++ b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorListSetHelper.scala @@ -0,0 +1,145 @@ +package com.twitter.lui.scrooge + +import com.twitter.algebird.Monoid +import com.twitter.lui.column_reader.BaseColumnReader +import com.twitter.lui.column_reader.cache.CacheColumnReader +import com.twitter.lui.column_reader.noncache.NonCacheColumnReader +import com.twitter.scrooge.ThriftEnum +import com.twitter.scrooge.{ ThriftStruct, ThriftUnion, ThriftStructField } +import com.twitter.scrooge.{ ThriftStructCodec3, ThriftStructFieldInfo } +import org.apache.parquet.column.page.PageReadStore +import org.apache.parquet.schema.{ MessageType, Type => ParquetSchemaType } +import org.apache.parquet.thrift.ThriftMetaData +import org.apache.thrift.protocol.TType +import scala.collection.JavaConverters._ +import scala.collection.mutable.{ Map => MMap } +import scala.reflect.ClassTag + +object ScroogeGeneratorListSetHelper { + import ScroogeGenerator._ + + def nonMethod(seqManifest: Manifest[_], + existingPath: ExistingPath, + cols: Array[BaseColumnReader], + mapData: Map[String, (ParquetFieldInfo, Option[Int], Option[ParquetThriftEnumInfo], Vector[Short])], + isOptional: Boolean, + isRequired: Boolean): (String, String) = { + + val returnType = seqManifest.runtimeClass.getName match { + case "scala.collection.Set" => "Set" + case "scala.collection.Seq" => "Vector" + case t => sys.error(s"Should not enter the List and Set helper with type $t") + } + + val path = existingPath.toStr + + val manifestF = seqManifest.typeArguments.head + val clazzF = manifestF.runtimeClass + val (optionalGroupFieldInfo, _, _, _) = mapData(path) + val (repeatedGroupFieldInfo, _, _, repeatedGroupPathV) = mapData(s"$path/1") + val fieldRepetitionLevel = repeatedGroupFieldInfo.maxRepetitionLevel + val fieldDefinitionLevel = repeatedGroupFieldInfo.maxDefinitionLevel + + // Handling the case here of a list of primitives, + // there is no sub child to ask about a closest primitive + val (_, Some(fieldClosestPrimitiveColumnIdx), _, _) = repeatedGroupFieldInfo.closestPrimitiveChild.map { pri => mapData(pri.toVector.mkString("/")) }.getOrElse(mapData(s"$path/1")) + + val (subTpeName, readColumn) = clazzF.getName match { + case "java.nio.ByteBuffer" => ("java.nio.ByteBuffer", "{primitive.primitiveAdvance(recordIdx, repetitionOffsets); primitive.getBinary()}") + case "java.lang.String" => ("String", "{primitive.primitiveAdvance(recordIdx, repetitionOffsets); primitive.getString()}") + case "boolean" => ("Boolean", "{primitive.primitiveAdvance(recordIdx, repetitionOffsets); primitive.getBoolean()}") + case "byte" => ("Byte", "{primitive.primitiveAdvance(recordIdx, repetitionOffsets); primitive.getInteger().toByte}") + case "double" => ("Double", "{primitive.primitiveAdvance(recordIdx, repetitionOffsets); primitive.getDouble()}") + case "int" => ("Int", "{primitive.primitiveAdvance(recordIdx, repetitionOffsets); primitive.getInteger()}") + case "long" => ("Long", "{primitive.primitiveAdvance(recordIdx, repetitionOffsets); primitive.getLong()}") + case "short" => ("Short", "{primitive.primitiveAdvance(recordIdx, repetitionOffsets); primitive.getInteger().toShort}") + case "scala.collection.Seq" => + val innerClassName = manifestF.typeArguments.head.runtimeClass.getName + (s"scala.collection.Seq[$innerClassName]", ScroogeGeneratorListSetHelper.nonMethod(manifestF, ExistingPath(repeatedGroupPathV.toVector.mkString("/")), cols, mapData, false, true)._2.split("\n").map(l => " " + l).mkString("\n")) + case name if classOf[ThriftEnum].isAssignableFrom(clazzF) => (name, s"""{primitive.primitiveAdvance(recordIdx, repetitionOffsets); ${name}enumDecode(primitive.getString())}""") + case name if classOf[ThriftUnion].isAssignableFrom(clazzF) => (name, s"""scroogeGenerator.buildUnion(classOf[$name], $name.fieldInfos, "${repeatedGroupPathV.toVector.mkString("/")}", colReaders, recordIdx, repetitionOffsets).asInstanceOf[$name]""") + case name if classOf[ThriftStruct].isAssignableFrom(clazzF) => (name, s"""scroogeGenerator.buildStruct(classOf[$name], $name.fieldInfos, "${repeatedGroupPathV.toVector.mkString("/")}", colReaders, recordIdx, repetitionOffsets).asInstanceOf[$name]""") + } + + // Here we need to handle the difference between None list and a Some(List()) + // we know the definition level for an optional for if the list is some or none will be up one level + val setStatement = (isOptional, isRequired) match { + case (true, _) => + s""" +if(elements > 0) + Some(listBuilder.result()) +else { + if(primitive.notNull(${optionalGroupFieldInfo.maxDefinitionLevel}, recordIdx,cachePos,stateOffests,repetitionOffsets)) + Some(${returnType}()) + else + None +}""" + case (_, false) => s""" +if(elements > 0) + listBuilder.result() + else { + if(primitive.notNull(${optionalGroupFieldInfo.maxDefinitionLevel}, recordIdx,cachePos,stateOffests,repetitionOffsets)) + ${returnType}() + else + null +}""" + case (_, true) => s"listBuilder.result()" + } + + val maxRepLevelOfPrimitive = cols(fieldClosestPrimitiveColumnIdx).maxRepetitionLevel + + ("", raw"""{ + val primitive = colReaders($fieldClosestPrimitiveColumnIdx).asInstanceOf[CacheColumnReader] + primitive.advanceSetRecord(recordIdx) + val listBuilder = ${returnType}.newBuilder[$subTpeName] + val cachePos = Array[Int](0) + val stateOffests = Array[Int](${(0 until maxRepLevelOfPrimitive + 1).map(_ => 0).mkString(",")}) + var elements: Int = 0 + + while(primitive.notNull($fieldDefinitionLevel, recordIdx,cachePos,stateOffests,repetitionOffsets)) { + val a = $readColumn + listBuilder += a + repetitionOffsets($fieldRepetitionLevel) += 1 + elements += 1 + } + repetitionOffsets($fieldRepetitionLevel) = 0 + + $setStatement + } + """) + } + + def apply(f: ThriftStructFieldInfo, existingPath: ExistingPath, + cols: Array[BaseColumnReader], + mapData: Map[String, (ParquetFieldInfo, Option[Int], Option[ParquetThriftEnumInfo], Vector[Short])], + fieldIdx: Int): (String, String) = { + + val path = ExistingPath(if (existingPath.toStr.isEmpty) f.id.toString else s"${existingPath.toStr}/${f.id}") + + val (objectSettings, methodContents) = nonMethod(f.manifest, path, cols, mapData, f.isOptional, f.isRequired) + + val returnType = f.toGenT match { + case SetT => "Set" + case ListT => "Vector" + case t => sys.error(s"Should not enter the List and Set helper with type $t") + } + val manifestF = f.manifest.typeArguments.head + val clazzF = manifestF.runtimeClass + val subTpeName = clazzF.getName match { + case "boolean" => "Boolean" + case "byte" => "Byte" + case "double" => "Double" + case "int" => "Int" + case "long" => "Long" + case "short" => "Short" + case "scala.collection.Seq" => + val innerClassName = manifestF.typeArguments.head.runtimeClass.getName + s"scala.collection.Seq[$innerClassName]" + case rest => rest + } + + val fieldTypeString = if (f.isOptional) s"Option[${returnType}[$subTpeName]]" else s"${returnType}[$subTpeName]" + (objectSettings, raw"""lazy val `${f.name}`: $fieldTypeString = $methodContents""") + } + +} diff --git a/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorMapHelper.scala b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorMapHelper.scala new file mode 100644 index 0000000000..79317ed99e --- /dev/null +++ b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorMapHelper.scala @@ -0,0 +1,123 @@ +package com.twitter.lui.scrooge + +import com.twitter.algebird.Monoid +import com.twitter.lui.column_reader.BaseColumnReader +import com.twitter.lui.column_reader.cache.CacheColumnReader +import com.twitter.lui.column_reader.noncache.NonCacheColumnReader +import com.twitter.scrooge.ThriftEnum +import com.twitter.scrooge.{ ThriftStruct, ThriftUnion, ThriftStructField } +import com.twitter.scrooge.{ ThriftStructCodec3, ThriftStructFieldInfo } +import org.apache.parquet.column.page.PageReadStore +import org.apache.parquet.schema.{ MessageType, Type => ParquetSchemaType } +import org.apache.parquet.thrift.ThriftMetaData +import org.apache.thrift.protocol.TType +import scala.collection.JavaConverters._ +import scala.collection.mutable.{ Map => MMap } +import scala.reflect.ClassTag + +object ScroogeGeneratorMapHelper { + import ScroogeGenerator._ + def clazzToTpeAndColumn(clazzF: Class[_], colName: String, fieldIdx: Int, innerIdx: Int, path: Vector[Short]) = + clazzF.getName match { + case "java.lang.String" => ("String", raw"{$colName.primitiveAdvance(recordIdx, repetitionOffsets); $colName.getString()}", true) + case "boolean" => ("Boolean", raw"{$colName.primitiveAdvance(recordIdx, repetitionOffsets); $colName.getBoolean()}", true) + case "byte" => ("Byte", raw"{$colName.primitiveAdvance(recordIdx, repetitionOffsets); $colName.getInteger().toByte}", true) + case "double" => ("Double", raw"{$colName.primitiveAdvance(recordIdx, repetitionOffsets); $colName.getDouble()}", true) + case "int" => ("Int", raw"{$colName.primitiveAdvance(recordIdx, repetitionOffsets); $colName.getInteger()}", true) + case "long" => ("Long", raw"{$colName.primitiveAdvance(recordIdx, repetitionOffsets); $colName.getLong()}", true) + case "short" => ("Short", raw"{$colName.primitiveAdvance(recordIdx, repetitionOffsets); $colName.getInteger().toShort}", true) + case name if classOf[ThriftEnum].isAssignableFrom(clazzF) => + (name, raw"""{$colName.primitiveAdvance(recordIdx, repetitionOffsets); enumDecode${fieldIdx}_${innerIdx}($colName.getString())}""", true) + case name if classOf[ThriftUnion].isAssignableFrom(clazzF) => + (name, raw"""scroogeGenerator.buildStruct(classOf[$name], $name.fieldInfos, "${path.mkString("/")}", colReaders, recordIdx, repetitionOffsets).asInstanceOf[$name]""", false) + case name if classOf[ThriftStruct].isAssignableFrom(clazzF) => + (name, raw"""scroogeGenerator.buildStruct(classOf[$name], $name.fieldInfos, "${path.mkString("/")}", colReaders, recordIdx, repetitionOffsets).asInstanceOf[$name]""", false) + } + + def apply(f: ThriftStructFieldInfo, existingPath: ExistingPath, + cols: Array[BaseColumnReader], + mapData: Map[String, (ParquetFieldInfo, Option[Int], Option[ParquetThriftEnumInfo], Vector[Short])], + fieldIdx: Int): (String, String) = { + + val path = if (existingPath.toStr.isEmpty) f.id.toString else s"${existingPath.toStr}/${f.id}" + + val (keyManifest, valueManifest) = (f.manifest.typeArguments(0), f.manifest.typeArguments(1)) + val (keyClazz, valueClazz) = (keyManifest.runtimeClass, valueManifest.runtimeClass) + + val (locFieldInfo, _, _, _) = mapData(path) + + val (keyFieldInfo, keyColumnOpt, _, pathK) = mapData(s"$path/1") + val (valueFieldInfo, valueColumnOpt, _, pathV) = mapData(s"$path/2") + + val fieldRepetitionLevel = keyFieldInfo.maxRepetitionLevel + val fieldDefinitionLevel = keyFieldInfo.maxDefinitionLevel + + // Handling the case here of a list of primitives, + // there is no sub child to ask about a closest primitive + val (_, Some(fieldClosestPrimitiveColumnIdx), _, _) = locFieldInfo.closestPrimitiveChild.map { pri => mapData(pri.toVector.mkString("/")) }.getOrElse(mapData(path)) + + val (keySubTpeName, keyReadColumn, keyIsPrimitive) = clazzToTpeAndColumn(keyClazz, "keyColumn", fieldIdx, 0, pathK) + val (valueSubTpeName, valueReadColumn, valueIsPrimitive) = clazzToTpeAndColumn(valueClazz, "valueColumn", fieldIdx, 1, pathV) + + val fieldTypeString = if (f.isOptional) s"Option[Map[$keySubTpeName, $valueSubTpeName]]" else s"Map[$keySubTpeName, $valueSubTpeName]" + + // Here we need to handle the difference between None list and a Some(List()) + // we know the definition level for an optional for if the list is some or none will be up one level + val setStatement = if (f.isOptional) s"""if(elements > 0) Some(mapBuilder.result()) else { + if(primitive.notNull(${fieldDefinitionLevel - 1}, recordIdx,cachePos,stateOffests,repetitionOffsets)) Some(Map()) else None + }""" + else s"mapBuilder.result()" + + val setupStr = (keyIsPrimitive, valueIsPrimitive) match { + case (true, true) => + s""" + val keyColumn = colReaders(${keyColumnOpt.get}).asInstanceOf[CacheColumnReader] + val valueColumn = colReaders(${valueColumnOpt.get}).asInstanceOf[CacheColumnReader] + val primitive = keyColumn + keyColumn.advanceSetRecord(recordIdx) + valueColumn.advanceSetRecord(recordIdx) + """ + case (true, false) => + s""" + val keyColumn = colReaders(${keyColumnOpt.get}).asInstanceOf[CacheColumnReader] + val primitive = keyColumn + keyColumn.advanceSetRecord(recordIdx) + """ + + case (false, true) => + s""" + val primitive = colReaders($fieldClosestPrimitiveColumnIdx).asInstanceOf[CacheColumnReader] + val valueColumn = colReaders(${valueColumnOpt.get}).asInstanceOf[CacheColumnReader] + primitive.advanceSetRecord(recordIdx) + valueColumn.advanceSetRecord(recordIdx) + """ + + case (false, false) => + s""" + val primitive = colReaders($fieldClosestPrimitiveColumnIdx).asInstanceOf[CacheColumnReader] + primitive.advanceSetRecord(recordIdx) + """ + } + + ("", raw""" + lazy val `${f.name}`: $fieldTypeString = { + $setupStr + val mapBuilder = Map.newBuilder[$keySubTpeName, $valueSubTpeName] + val cachePos = Array[Int](0) + val stateOffests = Array[Int](${(0 until fieldRepetitionLevel + 2).map(_ => 0).mkString(",")}) + var elements: Int = 0 + + while(primitive.notNull($fieldDefinitionLevel, recordIdx,cachePos,stateOffests,repetitionOffsets)) { + val a = ($keyReadColumn, $valueReadColumn) + mapBuilder += a + repetitionOffsets($fieldRepetitionLevel) += 1 + elements += 1 + } + repetitionOffsets($fieldRepetitionLevel) = 0 + + $setStatement + } + + """) + } +} diff --git a/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorPrimitiveHelper.scala b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorPrimitiveHelper.scala new file mode 100644 index 0000000000..cb9d7f1eee --- /dev/null +++ b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorPrimitiveHelper.scala @@ -0,0 +1,64 @@ +package com.twitter.lui.scrooge + +import com.twitter.algebird.Monoid +import com.twitter.lui.column_reader.BaseColumnReader +import com.twitter.lui.column_reader.cache.CacheColumnReader +import com.twitter.lui.column_reader.noncache.NonCacheColumnReader +import com.twitter.scrooge.ThriftEnum +import com.twitter.scrooge.{ ThriftStruct, ThriftUnion, ThriftStructField } +import com.twitter.scrooge.{ ThriftStructCodec3, ThriftStructFieldInfo } +import org.apache.parquet.column.page.PageReadStore +import org.apache.parquet.schema.{ MessageType, Type => ParquetSchemaType } +import org.apache.parquet.thrift.ThriftMetaData +import org.apache.thrift.protocol.TType +import scala.collection.JavaConverters._ +import scala.collection.mutable.{ Map => MMap } +import scala.reflect.ClassTag + +object ScroogeGeneratorPrimitiveHelper { + import ScroogeGenerator._ + + def nonMethod(f: ThriftStructFieldInfo, existingPath: ExistingPath, + cols: Array[BaseColumnReader], + mapData: Map[String, (ParquetFieldInfo, Option[Int], Option[ParquetThriftEnumInfo], Vector[Short])], + fieldIdx: Int): (String, String) = { + + val path = if (existingPath.toStr.isEmpty) f.id.toString else s"${existingPath.toStr}/${f.id}" + val primitiveZero = bytePrimtiveZero(f) + val getter = byteToJavaGetter(f) + + mapData.get(path).map { + case (parquetFieldInfo, Some(colIdx), enumInfoOpti, _) => + val setString = if (f.isOptional) + s"""if(isNull) { + None + } else { + Some($getter) + } + """ + else + s"if(isNull) $primitiveZero else $getter" + + ("", s"""{ + val rdr = colReaders($colIdx) + val isNull: Boolean = !rdr.primitiveAdvance(recordIdx, repetitionOffsets) + $setString + } + """) + case args @ _ => sys.error(s"Invalid args: $args") + }.getOrElse("", if (f.isOptional) "None" else primitiveZero) + } + + def apply(f: ThriftStructFieldInfo, existingPath: ExistingPath, + cols: Array[BaseColumnReader], + mapData: Map[String, (ParquetFieldInfo, Option[Int], Option[ParquetThriftEnumInfo], Vector[Short])], + fieldIdx: Int): (String, String) = { + val tpeString = byteToJavaPrimitive(f) + + val (objectSetup, innerBuilder) = nonMethod(f, existingPath, cols, mapData, fieldIdx) + + ("", s""" + lazy val `${f.name}`: $tpeString = $innerBuilder + """) + } +} diff --git a/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorStructBuilder.scala b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorStructBuilder.scala new file mode 100644 index 0000000000..7b40ce46e0 --- /dev/null +++ b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorStructBuilder.scala @@ -0,0 +1,103 @@ +package com.twitter.lui.scrooge + +import com.twitter.algebird.Monoid +import com.twitter.lui.column_reader.BaseColumnReader +import com.twitter.lui.column_reader.cache.CacheColumnReader +import com.twitter.lui.column_reader.noncache.NonCacheColumnReader +import com.twitter.scrooge.ThriftEnum +import com.twitter.scrooge.{ ThriftStruct, ThriftUnion, ThriftStructField } +import com.twitter.scrooge.{ ThriftStructCodec3, ThriftStructFieldInfo } +import org.apache.parquet.column.page.PageReadStore +import org.apache.parquet.schema.{ MessageType, Type => ParquetSchemaType } +import org.apache.parquet.thrift.ThriftMetaData +import org.apache.thrift.protocol.TType +import scala.collection.JavaConverters._ +import scala.collection.mutable.{ Map => MMap } +import scala.reflect.ClassTag + +object ScroogeGeneratorStructBuilder { + import ScroogeGenerator._ + + private[scrooge] def apply[T <: ThriftStruct](mapData: Map[String, (ParquetFieldInfo, Option[Int], Option[ParquetThriftEnumInfo], Vector[Short])], + mainClass: Class[T], + metadata: List[ThriftStructFieldInfo], + existingPath: ExistingPath, + cols: Array[BaseColumnReader])(implicit compiler: CustomCompiler): ((Array[BaseColumnReader], ScroogeGenerator, Long, Array[Int]) => T) = { + + val className = mainClass.getName + + val structFieldInfo = mapData(existingPath.toStr)._1 + val (closestPrimitiveFieldInfo, Some(closestPrimitiveColumnIdx), optiEnumInfo, _) = mapData(structFieldInfo.closestPrimitiveChild.get.toVector.mkString("/")) + + val repetitionLevel = structFieldInfo.maxRepetitionLevel + + val enumHelpers = ScroogeGeneratorEnumHelper(metadata, existingPath, cols, mapData) + + val (objectSections: String, fieldSections: String) = Monoid.sum(metadata.zipWithIndex.map { + case (f: ThriftStructFieldInfo, idx) => + + val (objectCode: String, builderStr: String) = f.toGenT match { + case PrimitiveT => ScroogeGeneratorPrimitiveHelper(f, existingPath, cols, mapData, idx) + + case StructT => ScroogeGeneratorStructHelper(f, existingPath, cols, mapData, idx) + + case ListT => ScroogeGeneratorListSetHelper(f, existingPath, cols, mapData, idx) + + case SetT => ScroogeGeneratorListSetHelper(f, existingPath, cols, mapData, idx) + + case MapT => ScroogeGeneratorMapHelper(f, existingPath, cols, mapData, idx) + } + + (objectCode, builderStr) + }) + + val repetitionCopy = if (structFieldInfo.maxRepetitionLevel > 0) + "Arrays.copyOf(repetitionOffsets, repetitionOffsets.length)" + else + "repetitionOffsets" + + val subclassName = s"ClassInst${scala.math.abs(className.hashCode)}" + + val code = s""" +import com.twitter.lui.column_reader.BaseColumnReader +import com.twitter.lui.column_reader.cache.CacheColumnReader +import com.twitter.lui.column_reader.noncache.NonCacheColumnReader +import com.twitter.lui.scrooge.ScroogeGenerator +import java.util.Arrays +import scala.collection.immutable.List +import scala.collection.immutable.Map +import scala.collection.immutable.Vector +import scala.collection.mutable.Builder +import scala.collection.mutable.MutableList + +object ${subclassName} { + $enumHelpers + $objectSections +} +class ${subclassName}(colReaders: Array[BaseColumnReader], scroogeGenerator: ScroogeGenerator, recordIdx: Long , repetitionOffsets: Array[Int]) extends $className { + import ${subclassName}._ + $fieldSections + + override lazy val hashCode: Int = super.hashCode +} +new Function4[Array[BaseColumnReader], ScroogeGenerator, Long, Array[Int], $className]{ + def apply(colReaders: Array[BaseColumnReader], scroogeGenerator: ScroogeGenerator , recordIdx: Long, repetitionOffsets: Array[Int]): $className = + new ${subclassName}(colReaders, scroogeGenerator, recordIdx, $repetitionCopy) +} + """ + // println(code) + // // to stash the generated classes: + // import java.io._ + // val pw = new PrintWriter(new File(s"/tmp/class_${className}.scala" )) + // pw.write(code) + // pw.close + try { + compiler.compile[Function4[Array[BaseColumnReader], ScroogeGenerator, Long, Array[Int], T]](code) + } catch { + case e: Throwable => + println(s"Failed when compiling:\n$code") + throw e + } + + } +} \ No newline at end of file diff --git a/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorStructHelper.scala b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorStructHelper.scala new file mode 100644 index 0000000000..a1ab5b83d2 --- /dev/null +++ b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorStructHelper.scala @@ -0,0 +1,82 @@ +package com.twitter.lui.scrooge + +import com.twitter.algebird.Monoid +import com.twitter.lui.column_reader.BaseColumnReader +import com.twitter.lui.column_reader.cache.CacheColumnReader +import com.twitter.lui.column_reader.noncache.NonCacheColumnReader +import com.twitter.scrooge.ThriftEnum +import com.twitter.scrooge.{ ThriftStruct, ThriftUnion, ThriftStructField } +import com.twitter.scrooge.{ ThriftStructCodec3, ThriftStructFieldInfo } +import org.apache.parquet.column.page.PageReadStore +import org.apache.parquet.schema.{ MessageType, Type => ParquetSchemaType } +import org.apache.parquet.thrift.ThriftMetaData +import org.apache.thrift.protocol.TType +import scala.collection.JavaConverters._ +import scala.collection.mutable.{ Map => MMap } +import scala.reflect.ClassTag + +object ScroogeGeneratorStructHelper { + import ScroogeGenerator._ + + def nonMethod(f: ThriftStructFieldInfo, existingPath: ExistingPath, + cols: Array[BaseColumnReader], + mapData: Map[String, (ParquetFieldInfo, Option[Int], Option[ParquetThriftEnumInfo], Vector[Short])], + fieldIdx: Int): (String, String) = { + val path = if (existingPath.toStr.isEmpty) f.id.toString else s"${existingPath.toStr}/${f.id}" + val tpeString = byteToJavaPrimitive(f) // Can include things like Option[String] + val tpeStringNoOption = byteToJavaPrimitive(f, false) // in the example above would be String instead + val nullV = if (f.isOptional) "None" else "null" + + mapData.get(path).map{ + case (locFieldInfo, _, _, pathV) => + val fieldRepetitionLevel = locFieldInfo.maxRepetitionLevel + val fieldDefinitionLevel = locFieldInfo.maxDefinitionLevel + val (_, Some(fieldClosestPrimitiveColumnIdx), _, _) = mapData(locFieldInfo.closestPrimitiveChild.get.toVector.mkString("/")) + + val buildType = if (classOf[ThriftUnion].isAssignableFrom(Class.forName(tpeStringNoOption))) "Union" else "Struct" + + val innerBuilder = s"""scroogeGenerator.build${buildType}(classOf[$tpeStringNoOption], $tpeStringNoOption.fieldInfos , "${pathV.mkString("/")}", colReaders, recordIdx, repetitionOffsets).asInstanceOf[$tpeStringNoOption]""" + + val someV = if (f.isOptional) + s"Some($innerBuilder)" + else + innerBuilder + + val notNullCall = if (fieldRepetitionLevel > 0) + s"""notNull($fieldDefinitionLevel, + recordIdx, + Array[Int](0), + Array[Int](${(0 until fieldRepetitionLevel + 2).map(_ => 0).mkString(",")}), + repetitionOffsets)""" + else + s"notNull($fieldDefinitionLevel, recordIdx)" + + ("", s"""{ + val primitive = colReaders($fieldClosestPrimitiveColumnIdx).asInstanceOf[CacheColumnReader] + primitive.advanceSetRecord(recordIdx) + + if(primitive.$notNullCall) { + $someV + } else { + $nullV + } + } + """) + }.getOrElse { + ("", s"$nullV") + } + } + + def apply(f: ThriftStructFieldInfo, existingPath: ExistingPath, + cols: Array[BaseColumnReader], + mapData: Map[String, (ParquetFieldInfo, Option[Int], Option[ParquetThriftEnumInfo], Vector[Short])], + fieldIdx: Int): (String, String) = { + + val (objectCode, innerGetCode) = nonMethod(f, existingPath, cols, mapData, fieldIdx) + val tpeString = byteToJavaPrimitive(f) // Can include things like Option[String] + + (objectCode, s""" + lazy val ${f.name}: $tpeString = $innerGetCode + """) + } +} diff --git a/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorUnionBuilder.scala b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorUnionBuilder.scala new file mode 100644 index 0000000000..5c48805bae --- /dev/null +++ b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ScroogeGeneratorUnionBuilder.scala @@ -0,0 +1,104 @@ +package com.twitter.lui.scrooge + +import com.twitter.algebird.Monoid +import com.twitter.lui.column_reader.BaseColumnReader +import com.twitter.lui.column_reader.cache.CacheColumnReader +import com.twitter.lui.column_reader.noncache.NonCacheColumnReader +import com.twitter.scrooge.ThriftEnum +import com.twitter.scrooge.{ ThriftStruct, ThriftUnion, ThriftStructField } +import com.twitter.scrooge.{ ThriftStructCodec3, ThriftUnionFieldInfo, ThriftStructFieldInfo } +import org.apache.parquet.column.page.PageReadStore +import org.apache.parquet.schema.{ MessageType, Type => ParquetSchemaType } +import org.apache.parquet.thrift.ThriftMetaData +import org.apache.thrift.protocol.TType +import scala.collection.JavaConverters._ +import scala.collection.mutable.{ Map => MMap } +import scala.reflect.ClassTag + +object ScroogeGeneratorUnionBuilder { + import ScroogeGenerator._ + + private[scrooge] def apply[T <: ThriftStruct](mapData: Map[String, (ParquetFieldInfo, Option[Int], Option[ParquetThriftEnumInfo], Vector[Short])], + mainClass: Class[T], + metadata: List[ThriftUnionFieldInfo[_, _]], + existingPath: ExistingPath, + cols: Array[BaseColumnReader])(implicit compiler: CustomCompiler): ((Array[BaseColumnReader], ScroogeGenerator, Long, Array[Int]) => T) = { + + val className = mainClass.getName + val structFieldInfo = mapData(existingPath.toStr)._1 + + val (objectSections: String, fieldSections: String) = Monoid.sum(metadata.zipWithIndex.map { + case (structUnionFieldInfo: ThriftUnionFieldInfo[_, _], idx) => + val structFieldInfo = structUnionFieldInfo.structFieldInfo + + val (objectCode: String, builderStr: String) = structFieldInfo.toGenT match { + case PrimitiveT => ScroogeGeneratorPrimitiveHelper.nonMethod(structFieldInfo, existingPath, cols, mapData, idx) + + case StructT => ScroogeGeneratorStructHelper.nonMethod(structFieldInfo, existingPath, cols, mapData, idx) + + // case ListT => ScroogeGeneratorListSetHelper(structFieldInfo, existingPath, cols, mapData, idx) + + // case SetT => ScroogeGeneratorListSetHelper(structFieldInfo, existingPath, cols, mapData, idx) + + // case MapT => ScroogeGeneratorMapHelper(structFieldInfo, existingPath, cols, mapData, idx) + } + + val path = if (existingPath.toStr.isEmpty) structFieldInfo.id.toString else s"${existingPath.toStr}/${structFieldInfo.id}" + val (pfi, optionPrimitiveCol, optParquetThriftEnumInfo, closestPrimitiveChild) = mapData(path) + val primitiveColumnIndex = optionPrimitiveCol.getOrElse(mapData(pfi.closestPrimitiveChild.get.toVector.mkString("/"))._2.get) + + val fieldNotNullTest = if (pfi.maxRepetitionLevel > 0) + s"""colReaders($primitiveColumnIndex).asInstanceOf[CacheColumnReader].notNull(${pfi.maxDefinitionLevel}, + recordIdx, + Array[Int](0), + Array[Int](${(0 until pfi.maxRepetitionLevel + 2).map(_ => 0).mkString(",")}), + repetitionOffsets)""" + else + s"colReaders($primitiveColumnIndex).asInstanceOf[CacheColumnReader].notNull(${pfi.maxDefinitionLevel}, recordIdx)" + + val classInstance = structUnionFieldInfo.fieldClassTag.runtimeClass.getName.replaceAll("\\$", ".") + val resultantBuilder = s"""if($fieldNotNullTest){ + val r = $builderStr + return $classInstance(r) + } + + """ + + (objectCode, resultantBuilder) + }) + + val code = s""" +import com.twitter.lui.column_reader.BaseColumnReader +import com.twitter.lui.column_reader.cache.CacheColumnReader +import com.twitter.lui.column_reader.noncache.NonCacheColumnReader +import com.twitter.lui.scrooge.ScroogeGenerator +import java.util.Arrays +import scala.collection.immutable.List +import scala.collection.immutable.Map +import scala.collection.immutable.Vector +import scala.collection.mutable.Builder +import scala.collection.mutable.MutableList + +new Function4[Array[BaseColumnReader], ScroogeGenerator, Long, Array[Int], $className]{ + def apply(colReaders: Array[BaseColumnReader], scroogeGenerator: ScroogeGenerator , recordIdx: Long, repetitionOffsets: Array[Int]): $className = { + $fieldSections + scala.sys.error("Unhandled field") + } +} + """ + // println(code) + // // to stash the generated classes: + // import java.io._ + // val pw = new PrintWriter(new File(s"/tmp/class_${className}.scala" )) + // pw.write(code) + // pw.close + try { + compiler.compile[Function4[Array[BaseColumnReader], ScroogeGenerator, Long, Array[Int], T]](code) + } catch { + case e: Throwable => + println(s"Failed when compiling:\n$code") + throw e + } + + } +} \ No newline at end of file diff --git a/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ThriftEnumMetadata.scala b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ThriftEnumMetadata.scala new file mode 100644 index 0000000000..935fcbe02c --- /dev/null +++ b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ThriftEnumMetadata.scala @@ -0,0 +1,52 @@ +package com.twitter.lui.scrooge + +import org.apache.parquet.thrift.struct.ThriftType._ + +import scala.collection.JavaConverters._ + +/** + * Creates a mapping from a path of thrift field IDs -> metadata about the enum at this position in the schema + * + * This is important because we need to know what the enums looked like at the time the file was written, so we can + * map them back into today's read schema (so that we allow renaming enums so long as their backing int doesn't change) + */ +case class ParquetThriftEnumInfo(m: Map[String, Int]) extends AnyVal + +object ThriftEnumMetadata extends StateVisitor[Map[ThriftIdPath, Map[String, Int]], Vector[Short]] { + + def get(fileDescriptor: StructType) = fileDescriptor.accept(this, Vector()) + + override def visit(enumType: EnumType, state: Vector[Short]): Map[ThriftIdPath, Map[String, Int]] = { + val m = enumType.getValues.iterator().asScala.map { e => + (e.getName, e.getId) + }.toMap + Map(ThriftIdPath(state) -> m) + } + + override def visit(mapType: MapType, state: Vector[Short]): Map[ThriftIdPath, Map[String, Int]] = { + val k = mapType.getKey.getType.accept(this, state :+ 1.toShort) + val v = mapType.getValue.getType.accept(this, state :+ 2.toShort) + k ++ v + } + + override def visit(setType: SetType, state: Vector[Short]): Map[ThriftIdPath, Map[String, Int]] = + setType.getValues.getType.accept(this, state :+ 1.toShort) + + override def visit(listType: ListType, state: Vector[Short]): Map[ThriftIdPath, Map[String, Int]] = + listType.getValues.getType.accept(this, state :+ 1.toShort) + + override def visit(structType: StructType, state: Vector[Short]): Map[ThriftIdPath, Map[String, Int]] = { + val children = structType.getChildren.asScala.map { c => + c.getType.accept(this, state :+ c.getFieldId) + } + children.reduce[Map[ThriftIdPath, Map[String, Int]]]{ case (m1, m2) => m1 ++ m2 } + } + + override def visit(boolType: BoolType, state: Vector[Short]): Map[ThriftIdPath, Map[String, Int]] = Map() + override def visit(byteType: ByteType, state: Vector[Short]): Map[ThriftIdPath, Map[String, Int]] = Map() + override def visit(doubleType: DoubleType, state: Vector[Short]): Map[ThriftIdPath, Map[String, Int]] = Map() + override def visit(i16Type: I16Type, state: Vector[Short]): Map[ThriftIdPath, Map[String, Int]] = Map() + override def visit(i32Type: I32Type, state: Vector[Short]): Map[ThriftIdPath, Map[String, Int]] = Map() + override def visit(i64Type: I64Type, state: Vector[Short]): Map[ThriftIdPath, Map[String, Int]] = Map() + override def visit(stringType: StringType, state: Vector[Short]): Map[ThriftIdPath, Map[String, Int]] = Map() +} diff --git a/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ThriftIdPathToParquetIndex.scala b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ThriftIdPathToParquetIndex.scala new file mode 100644 index 0000000000..0d013add2d --- /dev/null +++ b/lui-scrooge/src/main/scala/com/twitter/lui/scrooge/ThriftIdPathToParquetIndex.scala @@ -0,0 +1,175 @@ +package com.twitter.lui.scrooge + +import org.apache.parquet.schema.{ Type => ParquetSchemaType, MessageType, GroupType } +import org.apache.parquet.thrift.struct.ThriftType._ +import org.apache.parquet.thrift.struct.ThriftType +import org.apache.parquet.thrift.struct.ThriftField +import com.twitter.algebird.{ Monoid, Semigroup, MapAlgebra } +import scala.collection.JavaConverters._ + +case class PathToSchemaType(thriftIdPath: Vector[Short], + parquetIdPath: Vector[Int], + parquetStringPath: Vector[String], + parquetSchema: ParquetSchemaType, + insideCollection: Boolean) + +// A path of indices within groups to a field +case class ThriftIdPath(toVector: Vector[Short]) extends AnyVal + +case class ParquetFieldInfo(path: Vector[Int], + strPath: Vector[String], + parquetSchema: ParquetSchemaType, + closestPrimitiveChild: Option[ThriftIdPath], + maxDefinitionLevel: Int = 0, // These unfortunately aren't set in this file yet. Come in later + maxRepetitionLevel: Int = 0, + thriftType: ThriftType, + insideCollection: Boolean) { + def isPrimitive: Boolean = parquetSchema.isPrimitive +} + +case class ParquetMappingToThriftMapping(m: List[(ThriftIdPath, ParquetFieldInfo)]) extends AnyVal + +object ParquetMappingToThriftMappingMonoid extends Monoid[ParquetMappingToThriftMapping] { + def zero = ParquetMappingToThriftMapping(List()) + + private[this] def exceptionSemigroup[T] = new Semigroup[T] { + def plus(a: T, b: T) = sys.error(s"Fields $a and $b should not be combined. ExceptionSemigroup!.") + } + + def plus(a: ParquetMappingToThriftMapping, b: ParquetMappingToThriftMapping): ParquetMappingToThriftMapping = { + implicit val thriftIdSg = exceptionSemigroup[ParquetFieldInfo] + val combined = Monoid.plus(a.m, b.m) + require(combined.size == a.m.size + b.m.size) + ParquetMappingToThriftMapping(combined) + } +} + +/** + * Creates a mapping from thrift id path -> the index of the field in the parquet schema that was used + * to write the file. This is important, it allows us to reconcile the fact that parquet schema fields are keyed by name, + * but thrift fields are keyed by ID. So here we treat the id path as the source of truth, then figure out what that id path + * represented in the schema used to write the file. This lets us support renamed fields, as well as fields that are out of order + * in the IDL file, as well as fields that were inserted later out of order in the idl file. + */ +object ThriftIdPathToParquetIndex extends StateVisitor[ParquetMappingToThriftMapping, PathToSchemaType] { + implicit val pmtmMonoid = ParquetMappingToThriftMappingMonoid + def get(fileSchema: MessageType, fileDescriptor: StructType) = { + val recursive = fileDescriptor.accept(this, PathToSchemaType(Vector(), Vector(), Vector(), fileSchema, false)) + + val primitiveChild = Some(recursive.m.iterator.filter(_._2.isPrimitive).next._1) + + Monoid.plus( + ParquetMappingToThriftMapping( + List( + (ThriftIdPath(Vector()), ParquetFieldInfo(Vector(), Vector(), fileSchema, primitiveChild, 0, 0, fileDescriptor, false)))), + recursive) + } + + override def visit(structType: StructType, state: PathToSchemaType): ParquetMappingToThriftMapping = { + val groupType: GroupType = state.parquetSchema.asGroupType() + // Loop through the thrift fields coming from the side metadata + Monoid.sum(structType.getChildren.iterator.asScala.flatMap { c: ThriftField => + + val childIndex: Int = groupType.getFieldIndex(c.getName) + val parquetStringPath: Vector[String] = state.parquetStringPath :+ c.getName + val parquetIdPath: Vector[Int] = state.parquetIdPath :+ childIndex + + val matchingChild: ParquetSchemaType = groupType.getType(childIndex) + + val childPathToSchemaType = PathToSchemaType(state.thriftIdPath :+ c.getFieldId, parquetIdPath, parquetStringPath, matchingChild, state.insideCollection) + val childsMaps = c.getType.accept(this, childPathToSchemaType) + val primitiveChild = if (matchingChild.isPrimitive) None else Some(childsMaps.m.iterator.filter(_._2.isPrimitive).next._1) + + val childNodeValue = ParquetMappingToThriftMapping( + List(ThriftIdPath(childPathToSchemaType.thriftIdPath) -> ParquetFieldInfo(parquetIdPath, parquetStringPath, matchingChild, primitiveChild, 0, 0, c.getType, state.insideCollection))) + List(childNodeValue, childsMaps) + }) + } + + override def visit(mapType: MapType, state: PathToSchemaType): ParquetMappingToThriftMapping = { + val mapGroupType = state.parquetSchema.asGroupType() + + val groupType = mapGroupType.getType(0).asGroupType() + + val keyT = (groupType.getType(0), 1, mapType.getKey.getType) + val valueT = (groupType.getType(1), 2, mapType.getValue.getType) + + Monoid.sum(List(keyT, valueT).flatMap { + case (parquetCType, pathExtension, thriftInnerType) => + val parquetStringPath: Vector[String] = state.parquetStringPath ++ Vector("map", parquetCType.getName) + val parquetIdPath: Vector[Int] = state.parquetIdPath :+ pathExtension + + val childPathToSchemaType = PathToSchemaType(state.thriftIdPath :+ pathExtension.toShort, parquetIdPath, parquetStringPath, parquetCType, true) + val childsMaps = thriftInnerType.accept(this, childPathToSchemaType) + val primitiveChild = if (parquetCType.isPrimitive) None else Some(childsMaps.m.iterator.filter(_._2.isPrimitive).next._1) + + val childNodeValue = ParquetMappingToThriftMapping( + List(ThriftIdPath(childPathToSchemaType.thriftIdPath) -> ParquetFieldInfo(parquetIdPath, parquetStringPath, parquetCType, primitiveChild, 0, 0, thriftInnerType, true))) + List(childNodeValue, childsMaps) + }) + + } + + override def visit(setType: SetType, state: PathToSchemaType): ParquetMappingToThriftMapping = { + val groupType: GroupType = state.parquetSchema.asGroupType() + // Loop through the thrift fields coming from the side metadata + val c: ThriftType = setType.getValues.getType + val parquetCType = groupType.getType(0) + Monoid.sum(List(c).flatMap { c: ThriftType => + + val parquetStringPath: Vector[String] = state.parquetStringPath :+ parquetCType.getName + val parquetIdPath: Vector[Int] = state.parquetIdPath + + val childPathToSchemaType = PathToSchemaType(state.thriftIdPath :+ 1.toShort, parquetIdPath, parquetStringPath, parquetCType, true) + val childsMaps = c.accept(this, childPathToSchemaType) + val primitiveChild = if (parquetCType.isPrimitive) None else Some(childsMaps.m.iterator.filter(_._2.isPrimitive).next._1) + + val childNodeValue = ParquetMappingToThriftMapping( + List(ThriftIdPath(childPathToSchemaType.thriftIdPath) -> ParquetFieldInfo(parquetIdPath, parquetStringPath, parquetCType, primitiveChild, 0, 0, c, true))) + List(childNodeValue, childsMaps) + }) + } + + override def visit(listType: ListType, state: PathToSchemaType): ParquetMappingToThriftMapping = { + + val groupType: GroupType = state.parquetSchema.asGroupType() + // Loop through the thrift fields coming from the side metadata + val c: ThriftType = listType.getValues.getType + val parquetCType = groupType.getType(0) + Monoid.sum(List(c).flatMap { c: ThriftType => + + val parquetStringPath: Vector[String] = state.parquetStringPath :+ parquetCType.getName + val parquetIdPath: Vector[Int] = state.parquetIdPath + + val childPathToSchemaType = PathToSchemaType(state.thriftIdPath :+ 1.toShort, parquetIdPath, parquetStringPath, parquetCType, true) + val childsMaps = c.accept(this, childPathToSchemaType) + val primitiveChild = if (parquetCType.isPrimitive) None else Some(childsMaps.m.iterator.filter(_._2.isPrimitive).next._1) + + val childNodeValue = ParquetMappingToThriftMapping( + List(ThriftIdPath(childPathToSchemaType.thriftIdPath) -> ParquetFieldInfo(parquetIdPath, parquetStringPath, parquetCType, primitiveChild, 0, 0, c, true))) + List(childNodeValue, childsMaps) + }) + // val groupType = state.parquetSchema.asGroupType() + + // val childPathToSchemaType = PathToSchemaType(state.path :+ 1.toShort, child) + // val childsMaps = listType.getValues.getType.accept(this, childPathToSchemaType) + // childsMaps + } + + override def visit(enumType: EnumType, state: PathToSchemaType): ParquetMappingToThriftMapping = ParquetMappingToThriftMappingMonoid.zero + + override def visit(boolType: BoolType, state: PathToSchemaType): ParquetMappingToThriftMapping = ParquetMappingToThriftMappingMonoid.zero + + override def visit(byteType: ByteType, state: PathToSchemaType): ParquetMappingToThriftMapping = ParquetMappingToThriftMappingMonoid.zero + + override def visit(doubleType: DoubleType, state: PathToSchemaType): ParquetMappingToThriftMapping = ParquetMappingToThriftMappingMonoid.zero + + override def visit(i16Type: I16Type, state: PathToSchemaType): ParquetMappingToThriftMapping = ParquetMappingToThriftMappingMonoid.zero + + override def visit(i32Type: I32Type, state: PathToSchemaType): ParquetMappingToThriftMapping = ParquetMappingToThriftMappingMonoid.zero + + override def visit(i64Type: I64Type, state: PathToSchemaType): ParquetMappingToThriftMapping = ParquetMappingToThriftMappingMonoid.zero + + override def visit(stringType: StringType, state: PathToSchemaType): ParquetMappingToThriftMapping = ParquetMappingToThriftMappingMonoid.zero + +} diff --git a/lui-thrift-fixtures/src/test/resources/test.thrift b/lui-thrift-fixtures/src/test/resources/test.thrift new file mode 100644 index 0000000000..9b28d9ba42 --- /dev/null +++ b/lui-thrift-fixtures/src/test/resources/test.thrift @@ -0,0 +1,409 @@ +/** + * 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. + */ + +namespace java com.twitter.lui.thrift.thrift_java.test +#@namespace scala com.twitter.lui.thrift.thrift_scala.test + +struct TestListsInMap { + 1: string name, + 2: map,list> names, +} + +struct Name { + 1: required string first_name, + 2: optional string last_name, + 3: optional string type +} + +struct Address { + 1: string street, + 2: required string zip +} + +struct AddressWithStreetWithDefaultRequirement { + 1: string street, + 2: required string zip +} + +struct Phone { + 1: string mobile + 2: string work +} + +struct SinglePhone { + 1: string myOnlyPhone +} + +struct TestPerson { + 1: required Name name, + 2: optional i32 age, + 3: Address address, + 4: string info +} + + +struct RequiredMapFixture { + 1: optional string name, + 2: required map mavalue +} + +struct PrimitiveMap { + 1: required map test_bool, + 2: required map test_byte, + 3: required map test_i16, + 4: required map test_i32, + 5: required map test_i64, + 6: required map test_double, + 7: required map test_string +} + +struct PrimitiveList { + 1: required list test_bool, + 2: required list test_byte, + 3: required list test_i16, + 4: required list test_i32, + 5: required list test_i64, + 6: required list test_double, + 7: required list test_string +} + + +struct PrimitiveSet { + 1: required set test_bool, + 2: required set test_byte, + 3: required set test_i16, + 4: required set test_i32, + 5: required set test_i64, + 6: required set test_double, + 7: required set test_string +} + +struct MaybeBoolean { + 1: i64 id, + 2: optional bool maybe_bool +} + +struct StructWithOptionalInnerStructAndOptionalBooleanInside { + 1: optional MaybeBoolean booleanContainer +} + + + +struct StructWithOptionalInnerStruct { + 1: optional Phone ph +} + +struct NameList { + 1: required list names +} + +struct RequiredListFixture { + 1: optional string info, + 2: required list outerNames + #1: required list names +} + +struct RequiredSetFixture { + 1: optional string info, + 2: required set names +} + +struct RequiredPrimitiveFixture { + 1: required bool test_bool, + 2: required byte test_byte, + 3: required i16 test_i16, + 4: required i32 test_i32, + 5: required i64 test_i64, + 6: required double test_double, + 7: required string test_string, + 8: optional string info_string +} + + +struct StructWithReorderedOptionalFields { + 3: optional i32 fieldThree, + 2: optional i32 fieldTwo, + 1: optional i32 fieldOne, +} + +struct TestPersonWithRequiredPhone { + 1: required Name name, + 2: optional i32 age, + 3: Address address, + 4: string info, + 5: required Phone phone +} + +struct StructWithIndexStartsFrom4 { + 6: required Phone phone +} + +struct StructWithExtraField { + 3: required Phone extraPhone, + 6: required Phone phone +} + + +struct TestPersonWithRequiredPhone { + 1: required Name name, + 2: optional i32 age, + 3: required Address address, + 4: optional string info, + 5: required Phone phone +} + +struct TestPersonWithAllInformation { + 1: required Name name, + 2: optional i32 age, + 3: required Address address, + 4: optional Address working_address, + 5: optional string info, + 6: required map phone_map, + 7: optional set interests, + 8: optional list key_words +} + +struct EntryAB { + 1: required i32 id + 2: optional list primitiveList +} +struct EvtDetails { + 1: list entries +} +struct TestCaseNestedLists { + 1: optional list evtDetails +} + +struct TestMapComplex{ + 1: required map phone_address_map +} + +struct TestMapBinary{ + 1: required map string_binary_map +} + +struct TestMapPrimitiveKey { + 1: required map short_map, + 2: required map int_map, + 3: required map byt_map, + 4: required map bool_map, + 5: required map long_map, + 6: required map double_map, + 7: required map string_map; +} + +struct TestOptionalMap { + 1: optional map short_map, + 2: optional map int_map, + 3: optional map byt_map, + 4: optional map bool_map, + 5: optional map long_map, + 6: optional map double_map, + 7: optional map string_map +} + +struct TestListPrimitive { + 1: required list short_list, + 2: required list int_list, + 3: required list long_list, + 4: required list byte_list, + 5: required list string_list, + 6: required list bool_list, + 7: required list double_list, +} + +struct TestSetPrimitive { + 1: required set short_list, + 2: required set int_list, + 3: required set long_list, + 4: required set byte_list, + 5: required set string_list, + 6: required set bool_list, + 7: required set double_list +} + +struct TestMapPrimitiveValue { + 1: required map short_map, + 2: required map int_map, + 3: required map byt_map, + 4: required map bool_map, + 5: required map long_map, + 6: required map double_map, + 7: required map string_map +} + +struct SimpleListList { + 1: required list> list_of_lists +} + +union TestUnion { + 1: TestPerson first_person + 2: TestMapComplex second_map + 3: bool boolean_value +} + +enum Operation { + ADD = 1, + SUBTRACT = 2, + MULTIPLY = 3, + DIVIDE = 4 +} + + +struct RequiredEnumMapFixture { + 1: optional string name, + 2: required map mavalue +} + +struct TestFieldOfEnum{ + 1: required Operation op + 2: optional Operation op2 +} + +struct StringAndBinary { + 1: required string s + 2: required binary b +} + +#fixture fox nested structures +struct NestedList { + 1: required list> rll + 2: required list>> rlll + 3: optional list> oll + 4: optional list>> olll + 5: list> ll + 6: list>> lll +} + +struct ListNestMap { + 1: required list> rlm + 2: required list>> rllm + 3: optional list> olm + 4: optional list>> ollm + 5: list> lm + 6: list>> llm +} + +struct ListNestSet { + 1: required list> rls + 2: required list>> rlls + 3: optional list> ols + 4: optional list>> olls + 5: list> ls + 6: list>> lls +} + +struct ListNestEnum { + 1: required list rle +} + +struct MapNestMap { + 1: required map, map> rmm + 2: required map, Address>, map> rmmm + 3: optional map, map> omm + 4: optional map, Address>, map> ommm + 5: map, map> mm + 6: map, Address>, map> mmm +} + +struct MapNestList { + 1: required map, list
> rml + 2: required map>, list>> rmll + 3: optional map, list
> oml + 4: optional map>, list>> omll + 5: map, list
> ml + 6: map>, list>> mll +} + +struct MapNestSet { + 1: required map, set
> rms + 2: required map>, set>> rmss + 3: optional map, set
> oms + 4: optional map>, set>> omss + 5: map, set
> ms + 6: map>, set>> mss +} + +struct SetNestSet { + 1: required set> rss + 2: required set>> rsss + 3: optional set> oss + 4: optional set>> osss + 5: set> ss + 6: set>> sss +} + +struct SetNestList { + 1: required set> rsl + 2: required set>> rssl + 3: optional set> osl + 4: optional set>> ossl + 5: set> sl + 6: set>> ssl +} + +struct SetNestMap { + 1: required set> rsm + 2: required set>> rssm + 3: required set>>>> rssllm + 4: optional set> osm + 5: optional set>> ossm + 6: optional set>>>> ossllm + 7: set> sm + 8: set>> ssm + 9: set>>>> ssllm +} + +struct AString { + 1: required string s +} + +struct ALong { + 1: required i64 l +} + +struct ABool { + 1: required bool b +} + +struct AStructWithAStruct { + 1: required AString d +} + +union UnionV2 { + 1: AString aString, + 2: ALong aLong, + 3: ABool aNewBool +} + +struct StructWithUnionV2 { + 1: required string name, + 2: required UnionV2 aUnion +} + +struct AStructThatLooksLikeUnionV2 { + 1: optional AString aString, + 2: optional ALong aLong, + 3: optional ABool aNewBool +} + +struct StructWithAStructThatLooksLikeUnionV2 { + 1: required string name, + 2: required AStructThatLooksLikeUnionV2 aNotQuiteUnion +} diff --git a/project/Build.scala b/project/Build.scala index faaec2e027..a5f948fe85 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -244,7 +244,11 @@ object ScaldingBuild extends Build { scaldingJson, scaldingJdbc, maple, - scaldingSerialization + scaldingSerialization, + luiCore, + luiScrooge, + luiThriftFixtures, + luiScaldingScrooge ) lazy val formattingPreferences = { @@ -601,4 +605,75 @@ lazy val scaldingThriftMacros = module("thrift-macros") scaldingHadoopTest % "test", scaldingSerialization, scaldingThriftMacrosFixtures % "test->test") + + def luiModule(name: String) = { + val id = "lui-%s".format(name) + Project(id = id, base = file(id), settings = sharedSettings ++ Seq( + Keys.name := id)) + } + + lazy val luiCore = luiModule("core").settings( + libraryDependencies <++= (scalaVersion) { scalaVersion => + Seq( + "org.apache.parquet" % "parquet-column" % parquetVersion, + "org.apache.parquet" % "parquet-hadoop" % parquetVersion, + "org.slf4j" % "slf4j-api" % slf4jVersion, + "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", + "org.scala-lang" % "scala-reflect" % scalaVersion) + }) + + lazy val luiThriftFixtures = { + import ScroogeSBT.autoImport._ + luiModule("thrift-fixtures") + .settings(ScroogeSBT.newSettings: _*) + .settings( + scroogeThriftSourceFolder in Compile <<= baseDirectory { + base => base / "src/test/resources" + }, + sourceGenerators in Compile <+= ( + streams, + scroogeThriftSources in Compile, + scroogeIsDirty in Compile, + sourceManaged).map { (out, sources, isDirty, outputDir) => + // for some reason, sbt sometimes calls us multiple times, often with no source files. + if (isDirty && sources.nonEmpty) { + out.log.info("Generating scrooge thrift for %s ...".format(sources.mkString(", "))) + ScroogeSBT.compile(out.log, outputDir, sources.toSet, Set(), Map(), "java", Set()) + } + (outputDir ** "*.java").get.toSeq + }, + libraryDependencies ++= Seq( + "com.twitter" %% "scrooge-serializer" % scroogeVersion % "provided", + "org.apache.thrift" % "libthrift" % thriftVersion)) + } + + lazy val luiScrooge = luiModule("scrooge") + .settings( + libraryDependencies <++= (scalaVersion) { scalaVersion => + Seq( + "com.twitter" %% "algebird-core" % algebirdVersion, + "com.twitter" %% "scrooge-serializer" % scroogeVersion, + "org.slf4j" % "slf4j-api" % slf4jVersion, + "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", + "org.apache.parquet" % "parquet-column" % parquetVersion, + "org.apache.parquet" % "parquet-hadoop" % parquetVersion, + "org.apache.parquet" % "parquet-thrift" % parquetVersion, + "org.scala-lang" % "scala-compiler" % scalaVersion) + }) + .dependsOn(luiCore) + + lazy val luiScaldingScrooge = luiModule("scalding-scrooge") + .settings( + libraryDependencies ++= + Seq( + "org.slf4j" % "slf4j-api" % slf4jVersion, + "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", + "org.apache.parquet" % "parquet-column" % parquetVersion, + "org.apache.parquet" % "parquet-hadoop" % parquetVersion, + "org.apache.parquet" % "parquet-thrift" % parquetVersion, + "cascading" % "cascading-core" % cascadingVersion, + "cascading" % "cascading-hadoop" % cascadingVersion)) + .dependsOn(scaldingCore, luiScrooge, luiThriftFixtures % "test->test", scaldingParquetScrooge % "test->test") + } +