switch (schema.getCategory()) { case BOOLEAN: writer.value(((LongColumnVector) vector).vector[row] != 0); break; case BYTE: case INT: case LONG: writer.value(((LongColumnVector) vector).vector[row]); break; case FLOAT: case DOUBLE: writer.value(((DoubleColumnVector) vector).vector[row]); break; case STRING: case CHAR: case VARCHAR: writer.value(((BytesColumnVector) vector).toString(row)); break; case DECIMAL: writer.value(((DecimalColumnVector) vector).vector[row] .toString()); break; case DATE: writer.value(new DateWritable( (int) ((LongColumnVector) vector).vector[row]) .toString()); break; case TIMESTAMP:
/** * Append a double value. * @param d A double. * @return this * @throws JSONException If the number is not finite. */ public JSONWriter value(double d) throws JSONException { return this.value(new Double(d)); }
/** * Append a double value. * @param d A double. * @return this * @throws JSONException If the number is not finite. */ public JSONWriter value(double d) throws JSONException { return this.value(new Double(d)); }
private static void writeBloomFilterStats(JSONWriter writer, BloomFilterIO bf) throws JSONException { int bitCount = bf.getBitSize(); int popCount = 0; for (long l : bf.getBitSet()) { popCount += Long.bitCount(l); } int k = bf.getNumHashFunctions(); float loadFactor = (float) popCount / (float) bitCount; float expectedFpp = (float) Math.pow(loadFactor, k); writer.key("numHashFunctions").value(k); writer.key("bitCount").value(bitCount); writer.key("popCount").value(popCount); writer.key("loadFactor").value(loadFactor); writer.key("expectedFpp").value(expectedFpp); }
static void printBinary(JSONWriter writer, BytesColumnVector vector, int row) throws JSONException { writer.array(); int offset = vector.start[row]; for(int i=0; i < vector.length[row]; ++i) { writer.value(0xff & (int) vector.vector[row][offset + i]); } writer.endArray(); } static void printValue(JSONWriter writer, ColumnVector vector,
private static void writeStripeInformation(JSONWriter writer, StripeInformation stripe) throws JSONException { writer.object(); writer.key("offset").value(stripe.getOffset()); writer.key("indexLength").value(stripe.getIndexLength()); writer.key("dataLength").value(stripe.getDataLength()); writer.key("footerLength").value(stripe.getFooterLength()); writer.key("rowCount").value(stripe.getNumberOfRows()); writer.endObject(); }
OrcProto.Type type = types.get(typeId); if (obj == null) { writer.value(null); } else { switch (type.getKind()) { break; case BYTE: writer.value(((ByteWritable) obj).get()); break; case SHORT: writer.value(((ShortWritable) obj).get()); break; case INT: writer.value(((IntWritable) obj).get()); break; case LONG: writer.value(((LongWritable) obj).get()); break; case FLOAT: writer.value(((FloatWritable) obj).get()); break; case DOUBLE: writer.value(((DoubleWritable) obj).get()); break; case BOOLEAN: writer.value(((BooleanWritable) obj).get()); break; default:
switch (schema.getCategory()) { case BOOLEAN: writer.value(((LongColumnVector) vector).vector[row] != 0); break; case BYTE: case INT: case LONG: writer.value(((LongColumnVector) vector).vector[row]); break; case FLOAT: case DOUBLE: writer.value(((DoubleColumnVector) vector).vector[row]); break; case STRING: case CHAR: case VARCHAR: writer.value(((BytesColumnVector) vector).toString(row)); break; case BINARY: break; case DECIMAL: writer.value(((DecimalColumnVector) vector).vector[row].toString()); break; case DATE: writer.value(new DateWritable( (int) ((LongColumnVector) vector).vector[row]).toString()); break;
private static void writeColumnStatistics(JSONWriter writer, ColumnStatistics cs) throws JSONException { if (cs != null) { writer.key("count").value(cs.getNumberOfValues()); writer.key("hasNull").value(cs.hasNull()); if (cs instanceof BinaryColumnStatistics) { writer.key("totalLength").value(((BinaryColumnStatistics) cs).getSum()); writer.key("type").value(OrcProto.Type.Kind.BINARY); } else if (cs instanceof BooleanColumnStatistics) { writer.key("trueCount").value(((BooleanColumnStatistics) cs).getTrueCount()); writer.key("falseCount").value(((BooleanColumnStatistics) cs).getFalseCount()); writer.key("type").value(OrcProto.Type.Kind.BOOLEAN); } else if (cs instanceof IntegerColumnStatistics) { writer.key("min").value(((IntegerColumnStatistics) cs).getMinimum()); writer.key("max").value(((IntegerColumnStatistics) cs).getMaximum()); if (((IntegerColumnStatistics) cs).isSumDefined()) { writer.key("sum").value(((IntegerColumnStatistics) cs).getSum()); writer.key("type").value(OrcProto.Type.Kind.LONG); } else if (cs instanceof DoubleColumnStatistics) { writer.key("min").value(((DoubleColumnStatistics) cs).getMinimum()); writer.key("max").value(((DoubleColumnStatistics) cs).getMaximum()); writer.key("sum").value(((DoubleColumnStatistics) cs).getSum()); writer.key("type").value(OrcProto.Type.Kind.DOUBLE); } else if (cs instanceof StringColumnStatistics) { writer.key("min").value(((StringColumnStatistics) cs).getMinimum()); writer.key("max").value(((StringColumnStatistics) cs).getMaximum()); writer.key("totalLength").value(((StringColumnStatistics) cs).getSum()); writer.key("type").value(OrcProto.Type.Kind.STRING); } else if (cs instanceof DateColumnStatistics) {
for(OrcProto.Type type : types) { writer.object(); writer.key("columnId").value(i++); writer.key("columnType").value(type.getKind()); if (type.getFieldNamesCount() > 0) { writer.key("childColumnNames").array(); writer.key("precision").value(type.getPrecision()); writer.key("scale").value(type.getScale()); writer.key("maxLength").value(type.getMaximumLength());
writer.key("fileName").value(filename); Path path = new Path(filename); Reader reader = FileDump.getReader(path, conf, null); if (reader == null) { writer.key("status").value("FAILED"); continue; writer.key("fileVersion").value(reader.getFileVersion().getName()); writer.key("writerVersion").value(reader.getWriterVersion()); RecordReaderImpl rows = (RecordReaderImpl) reader.rows(); writer.key("numberOfRows").value(reader.getNumberOfRows()); writer.key("compression").value(reader.getCompressionKind()); if (reader.getCompressionKind() != CompressionKind.NONE) { writer.key("compressionBufferSize").value(reader.getCompressionSize()); writer.key("schemaString").value(reader.getSchema().toString()); writer.key("schema").array(); writeSchema(writer, reader.getTypes()); for (int n = 0; n < stripeStatistics.size(); n++) { writer.object(); writer.key("stripeNumber").value(n + 1); StripeStatistics ss = stripeStatistics.get(n); writer.key("columnStatistics").array(); for (int i = 0; i < ss.getColumnStatistics().length; i++) { writer.object(); writer.key("columnId").value(i); writeColumnStatistics(writer, ss.getColumnStatistics()[i]); writer.endObject();
private static void writeRowGroupIndexes(JSONWriter writer, int col, OrcProto.RowIndex[] rowGroupIndex) throws JSONException { OrcProto.RowIndex index; if (rowGroupIndex == null || (col >= rowGroupIndex.length) || ((index = rowGroupIndex[col]) == null)) { return; } writer.key("rowGroupIndexes").array(); for (int entryIx = 0; entryIx < index.getEntryCount(); ++entryIx) { writer.object(); writer.key("entryId").value(entryIx); OrcProto.RowIndexEntry entry = index.getEntry(entryIx); if (entry == null) { continue; } OrcProto.ColumnStatistics colStats = entry.getStatistics(); writeColumnStatistics(writer, ColumnStatisticsImpl.deserialize(colStats)); writer.key("positions").array(); for (int posIx = 0; posIx < entry.getPositionsCount(); ++posIx) { writer.value(entry.getPositions(posIx)); } writer.endArray(); writer.endObject(); } writer.endArray(); }
private static void writeBloomFilterIndexes(JSONWriter writer, int col, OrcProto.BloomFilterIndex[] bloomFilterIndex) throws JSONException { BloomFilterIO stripeLevelBF = null; if (bloomFilterIndex != null && bloomFilterIndex[col] != null) { int entryIx = 0; writer.key("bloomFilterIndexes").array(); for (OrcProto.BloomFilter bf : bloomFilterIndex[col].getBloomFilterList()) { writer.object(); writer.key("entryId").value(entryIx++); BloomFilterIO toMerge = new BloomFilterIO(bf); writeBloomFilterStats(writer, toMerge); if (stripeLevelBF == null) { stripeLevelBF = toMerge; } else { stripeLevelBF.merge(toMerge); } writer.endObject(); } writer.endArray(); } if (stripeLevelBF != null) { writer.key("stripeLevelBloomFilter"); writer.object(); writeBloomFilterStats(writer, stripeLevelBF); writer.endObject(); } }