diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetIndexPageFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetIndexPageFilter.java new file mode 100644 index 000000000000..32932721d7ee --- /dev/null +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetIndexPageFilter.java @@ -0,0 +1,516 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.parquet; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import org.apache.commons.compress.utils.Lists; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Binder; +import org.apache.iceberg.expressions.BoundReference; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionVisitors; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.util.ByteBuffers; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.internal.column.columnindex.ColumnIndex; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.internal.filter2.columnindex.RowRanges; +import org.apache.parquet.internal.filter2.columnindex.RowRanges.Range; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ParquetIndexPageFilter { + private static final Logger LOG = LoggerFactory.getLogger(ParquetIndexPageFilter.class); + + private final Schema schema; + private final MessageType fileSchema; + private final Expression expr; + + public ParquetIndexPageFilter(Schema schema, MessageType fileSchema, Expression expr, boolean caseSensitive) { + this.schema = schema; + this.fileSchema = fileSchema; + this.expr = Binder.bind(schema.asStruct(), Expressions.rewriteNot(expr), caseSensitive); + } + + public long applyIndex(ParquetFileReader reader, int rowGroupIndex) { + RowRanges ranges = new EvalVisitor(reader.getRowGroups().get(rowGroupIndex), reader).eval(); + ParquetRanges.setRanges(reader, rowGroupIndex, ranges); + return ranges.rowCount(); + } + + private class EvalVisitor extends ExpressionVisitors.BoundExpressionVisitor { + private final ParquetFileReader reader; + private final long totalRowCount; + private final RowRanges allRows; + private final Map cols = Maps.newHashMap(); + private final Map indexes = Maps.newHashMap(); + private final Map offsets = Maps.newHashMap(); + private final Map parquetTypes = Maps.newHashMap(); + private final Map icebergTypes = Maps.newHashMap(); + + private EvalVisitor(BlockMetaData rowGroup, ParquetFileReader reader) { + this.reader = reader; + this.totalRowCount = rowGroup.getRowCount(); + this.allRows = ParquetRanges.of(0, totalRowCount - 1); + + for (ColumnDescriptor desc : fileSchema.getColumns()) { + PrimitiveType colType = fileSchema.getType(desc.getPath()).asPrimitiveType(); + if (colType.getId() != null) { + int id = colType.getId().intValue(); + parquetTypes.put(id, colType); + icebergTypes.put(id, schema.findType(id).asPrimitiveType()); + } + } + + for (ColumnChunkMetaData meta : rowGroup.getColumns()) { + PrimitiveType colType = fileSchema.getType(meta.getPath().toArray()).asPrimitiveType(); + if (colType.getId() != null) { + int id = colType.getId().intValue(); + cols.put(id, meta); + } + } + } + + private RowRanges eval() { + return ExpressionVisitors.visit(expr, this); + } + + private Range asRange(int fieldId, int pageIndex) { + OffsetIndex offsets = offsets(fieldId); + if (offsets != null) { + return ParquetRanges.rangeOf( + offsets.getFirstRowIndex(pageIndex), + offsets.getLastRowIndex(pageIndex, totalRowCount)); + } else { + return ParquetRanges.rangeOf(0, totalRowCount - 1); + } + } + + @Override + public RowRanges isNull(BoundReference ref) { + int fieldId = ref.fieldId(); + ColumnIndex index = index(fieldId); + List nullCounts = index.getNullCounts(); + + List pageRanges = Lists.newArrayList(); + for (int i = 0; i < nullCounts.size(); i += 1) { + if (nullCounts.get(i) >= 0) { + // there is at least one null value in the page + pageRanges.add(asRange(fieldId, i)); + } + } + + return ParquetRanges.of(pageRanges); + } + + @Override + public RowRanges notNull(BoundReference ref) { + int fieldId = ref.fieldId(); + ColumnIndex index = index(fieldId); + List containsOnlyNull = index.getNullPages(); + + List pageRanges = Lists.newArrayList(); + for (int i = 0; i < containsOnlyNull.size(); i += 1) { + if (!containsOnlyNull.get(i)) { + // there is at least one non-null value in the page + pageRanges.add(asRange(fieldId, i)); + } + } + + return ParquetRanges.of(pageRanges); + } + + @Override + public RowRanges isNaN(BoundReference ref) { + return allRows; + } + + @Override + public RowRanges notNaN(BoundReference ref) { + return allRows; + } + + @Override + public RowRanges lt(BoundReference ref, Literal lit) { + int fieldId = ref.fieldId(); + ColumnIndex index = index(fieldId); + if (index == null) { + return allRows; + } + + List containsOnlyNull = index.getNullPages(); + List lowerBounds = index.getMinValues(); + + List pageRanges = Lists.newArrayList(); + for (int i = 0; i < containsOnlyNull.size(); i += 1) { + if (containsOnlyNull.get(i)) { + T lower = fromBytes(lowerBounds.get(i), parquetTypes.get(fieldId), icebergTypes.get(fieldId)); + if (lit.comparator().compare(lower, lit.value()) < 0) { + pageRanges.add(asRange(fieldId, i)); + } + } + } + + return ParquetRanges.of(pageRanges); + } + + @Override + public RowRanges ltEq(BoundReference ref, Literal lit) { + int fieldId = ref.fieldId(); + ColumnIndex index = index(fieldId); + if (index == null) { + return allRows; + } + + List containsOnlyNull = index.getNullPages(); + List lowerBounds = index.getMinValues(); + + List pageRanges = Lists.newArrayList(); + for (int i = 0; i < containsOnlyNull.size(); i += 1) { + if (!containsOnlyNull.get(i)) { + T lower = fromBytes(lowerBounds.get(i), parquetTypes.get(fieldId), icebergTypes.get(fieldId)); + if (lit.comparator().compare(lower, lit.value()) <= 0) { + pageRanges.add(asRange(fieldId, i)); + } + } + } + + return ParquetRanges.of(pageRanges); + } + + @Override + public RowRanges gt(BoundReference ref, Literal lit) { + int fieldId = ref.fieldId(); + ColumnIndex index = index(fieldId); + if (index == null) { + return allRows; + } + + List containsOnlyNull = index.getNullPages(); + List upperBounds = index.getMaxValues(); + + List pageRanges = Lists.newArrayList(); + for (int i = 0; i < containsOnlyNull.size(); i += 1) { + if (!containsOnlyNull.get(i)) { + T upper = fromBytes(upperBounds.get(i), parquetTypes.get(fieldId), icebergTypes.get(fieldId)); + if (lit.comparator().compare(upper, lit.value()) > 0) { + pageRanges.add(asRange(fieldId, i)); + } + } + } + + return ParquetRanges.of(pageRanges); + } + + @Override + public RowRanges gtEq(BoundReference ref, Literal lit) { + int fieldId = ref.fieldId(); + ColumnIndex index = index(fieldId); + if (index == null) { + return allRows; + } + + List containsOnlyNull = index.getNullPages(); + List upperBounds = index.getMaxValues(); + + List pageRanges = Lists.newArrayList(); + for (int i = 0; i < containsOnlyNull.size(); i += 1) { + if (!containsOnlyNull.get(i)) { + T upper = fromBytes(upperBounds.get(i), parquetTypes.get(fieldId), icebergTypes.get(fieldId)); + if (lit.comparator().compare(upper, lit.value()) >= 0) { + pageRanges.add(asRange(fieldId, i)); + } + } + } + + return ParquetRanges.of(pageRanges); + } + + @Override + public RowRanges eq(BoundReference ref, Literal lit) { + int fieldId = ref.fieldId(); + ColumnIndex index = index(fieldId); + if (index == null) { + return allRows; + } + + List containsOnlyNull = index.getNullPages(); + List lowerBounds = index.getMinValues(); + List upperBounds = index.getMaxValues(); + + List pageRanges = Lists.newArrayList(); + for (int i = 0; i < containsOnlyNull.size(); i += 1) { + if (!containsOnlyNull.get(i)) { + Comparator comparator = lit.comparator(); + T lower = fromBytes(lowerBounds.get(i), parquetTypes.get(fieldId), icebergTypes.get(fieldId)); + T upper = fromBytes(upperBounds.get(i), parquetTypes.get(fieldId), icebergTypes.get(fieldId)); + if (comparator.compare(lower, lit.value()) <= 0 && comparator.compare(lit.value(), upper) <= 0) { + pageRanges.add(asRange(fieldId, i)); + } + } + } + + return ParquetRanges.of(pageRanges); + } + + @Override + public RowRanges notEq(BoundReference ref, Literal lit) { + int fieldId = ref.fieldId(); + ColumnIndex index = index(fieldId); + if (index == null) { + return allRows; + } + + List nullCounts = index.getNullCounts(); + List lowerBounds = index.getMinValues(); + List upperBounds = index.getMaxValues(); + + List pageRanges = Lists.newArrayList(); + for (int i = 0; i < nullCounts.size(); i += 1) { + if (nullCounts.get(i) > 0) { + pageRanges.add(asRange(fieldId, i)); + continue; + } + + Comparator comparator = lit.comparator(); + T lower = fromBytes(lowerBounds.get(i), parquetTypes.get(fieldId), icebergTypes.get(fieldId)); + T upper = fromBytes(upperBounds.get(i), parquetTypes.get(fieldId), icebergTypes.get(fieldId)); + if (comparator.compare(lower, lit.value()) != 0 || comparator.compare(lit.value(), upper) != 0) { + pageRanges.add(asRange(fieldId, i)); + } + } + + return ParquetRanges.of(pageRanges); + } + + @Override + public RowRanges in(BoundReference ref, Set literalSet) { + int fieldId = ref.fieldId(); + ColumnIndex index = index(fieldId); + if (index == null) { + return allRows; + } + + List containsOnlyNull = index.getNullPages(); + List lowerBounds = index.getMinValues(); + List upperBounds = index.getMaxValues(); + + List pageRanges = Lists.newArrayList(); + for (int i = 0; i < containsOnlyNull.size(); i += 1) { + if (!containsOnlyNull.get(i)) { + Comparator comparator = ref.comparator(); + boolean overlapsRange = false; + for (T value : literalSet) { + T lower = fromBytes(lowerBounds.get(i), parquetTypes.get(fieldId), icebergTypes.get(fieldId)); + T upper = fromBytes(upperBounds.get(i), parquetTypes.get(fieldId), icebergTypes.get(fieldId)); + if (comparator.compare(lower, value) <= 0 && comparator.compare(value, upper) <= 0) { + overlapsRange = true; + } + } + + if (overlapsRange) { + pageRanges.add(asRange(fieldId, i)); + } + } + } + + return ParquetRanges.of(pageRanges); + } + + @Override + public RowRanges notIn(BoundReference ref, Set literalSet) { + return allRows; + } + + @Override + public RowRanges startsWith(BoundReference ref, Literal lit) { + return allRows; + } + + @Override + public RowRanges notStartsWith(BoundReference ref, Literal lit) { + return allRows; + } + + @Override + public RowRanges alwaysTrue() { + return allRows; + } + + @Override + public RowRanges alwaysFalse() { + return ParquetRanges.empty(); + } + + @Override + public RowRanges and(RowRanges left, RowRanges right) { + return ParquetRanges.intersection(left, right); + } + + @Override + public RowRanges or(RowRanges left, RowRanges right) { + return ParquetRanges.union(left, right); + } + + private ColumnIndex index(int id) { + ColumnIndex index = indexes.get(id); + if (index == null) { + ColumnChunkMetaData col = cols.get(id); + try { + index = reader.readColumnIndex(col); + indexes.put(id, index); + } catch (IOException e) { + LOG.warn("Failed to read column index for column: {} (skipping index)", id); + } + } + + return index; + } + + private OffsetIndex offsets(int id) { + OffsetIndex index = offsets.get(id); + if (index == null) { + ColumnChunkMetaData col = cols.get(id); + try { + index = reader.readOffsetIndex(col); + offsets.put(id, index); + } catch (IOException e) { + LOG.warn("Failed to read column index for column: {} (skipping index)", id); + } + } + + return index; + } + } + + @SuppressWarnings("unchecked") + private T fromBytes(ByteBuffer bytes, PrimitiveType primitiveType, Type.PrimitiveType icebergType) { + Optional converted = primitiveType.getLogicalTypeAnnotation() + .accept(new LogicalTypeAnnotation.LogicalTypeAnnotationVisitor() { + @Override + public Optional visit(LogicalTypeAnnotation.StringLogicalTypeAnnotation stringLogicalType) { + return Optional.of(StandardCharsets.UTF_8.decode(bytes)); + } + + @Override + public Optional visit(LogicalTypeAnnotation.EnumLogicalTypeAnnotation enumLogicalType) { + return Optional.of(StandardCharsets.UTF_8.decode(bytes)); + } + + @Override + public Optional visit(LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalType) { + switch (primitiveType.getPrimitiveTypeName()) { + case INT32: + return Optional.of(new BigDecimal(BigInteger.valueOf(bytes.getInt(0)), decimalType.getScale())); + case INT64: + return Optional.of(new BigDecimal(BigInteger.valueOf(bytes.getLong(0)), decimalType.getScale())); + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + new BigDecimal(new BigInteger(ByteBuffers.toByteArray(bytes)), decimalType.getScale()); + } + return Optional.empty(); + } + + @Override + public Optional visit(LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeLogicalType) { + switch (timeLogicalType.getUnit()) { + case MILLIS: + return Optional.of(((long) bytes.getInt(0)) * 1000L); + case MICROS: + return Optional.of(bytes.getLong(0)); + case NANOS: + return Optional.of(Math.floorDiv(bytes.getLong(0), 1000)); + } + return Optional.empty(); + } + + @Override + public Optional visit(LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampLogicalType) { + switch (timestampLogicalType.getUnit()) { + case MILLIS: + return Optional.of(bytes.getLong(0) * 1000); + case MICROS: + return Optional.of(bytes.getLong(0)); + case NANOS: + return Optional.of(Math.floorDiv(bytes.getLong(0), 1000)); + } + return Optional.empty(); + } + + @Override + public Optional visit(LogicalTypeAnnotation.JsonLogicalTypeAnnotation jsonLogicalType) { + return Optional.of(StandardCharsets.UTF_8.decode(bytes)); + } + + @Override + public Optional visit(LogicalTypeAnnotation.UUIDLogicalTypeAnnotation uuidLogicalType) { + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(uuidLogicalType); + } + }); + + if (converted.isPresent()) { + return (T) converted.get(); + } + + switch (primitiveType.getPrimitiveTypeName()) { + case BOOLEAN: + return (T) (Boolean) (bytes.get() != 0); + case INT32: + Integer intValue = bytes.getInt(); + if (icebergType.typeId() == Type.TypeID.LONG) { + return (T) (Long) intValue.longValue(); + } + return (T) intValue; + case INT64: + return (T) (Long) bytes.getLong(0); + case FLOAT: + Float floatValue = bytes.getFloat(0); + if (icebergType.typeId() == Type.TypeID.DOUBLE) { + return (T) (Double) floatValue.doubleValue(); + } + return (T) floatValue; + case DOUBLE: + return (T) (Double) bytes.getDouble(0); + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + return (T) bytes; + default: + throw new UnsupportedOperationException("Unsupported Parquet type: " + primitiveType); + } + } +} diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetRanges.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetRanges.java new file mode 100644 index 000000000000..7c6b7c2f239d --- /dev/null +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetRanges.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.parquet; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.common.DynConstructors; +import org.apache.iceberg.common.DynFields; +import org.apache.iceberg.common.DynMethods; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.internal.filter2.columnindex.RowRanges; + +/** + * Helper methods for creating and merging Parquet {@link RowRanges}. + */ +public class ParquetRanges { + private ParquetRanges() { + } + + private static final DynConstructors.Ctor RANGE_CTOR = DynConstructors.builder() + .hiddenImpl(RowRanges.Range.class, Long.TYPE, Long.TYPE) + .build(); + + public static RowRanges.Range rangeOf(long start, long end) { + return RANGE_CTOR.newInstance(start, end); + } + + private static final DynConstructors.Ctor RANGES_LIST_CTOR = DynConstructors.builder() + .hiddenImpl(RowRanges.class, List.class) + .build(); + + private static final RowRanges EMPTY = RANGES_LIST_CTOR.newInstance(ImmutableList.of()); + + public static RowRanges empty() { + return EMPTY; + } + + public static RowRanges of(long start, long end) { + return of(rangeOf(start, end)); + } + + public static RowRanges of(RowRanges.Range... ranges) { + return of(Lists.newArrayList(ranges)); + } + + public static RowRanges of(List ranges) { + return RANGES_LIST_CTOR.newInstance(ranges); + } + + private static final DynMethods.StaticMethod UNION = DynMethods.builder("union") + .impl(RowRanges.class, RowRanges.class, RowRanges.class) + .buildStatic(); + + public static RowRanges union(RowRanges left, RowRanges right) { + return UNION.invoke(left, right); + } + + private static final DynMethods.StaticMethod INTERSECTION = DynMethods.builder("intersection") + .impl(RowRanges.class, RowRanges.class, RowRanges.class) + .buildStatic(); + + public static RowRanges intersection(RowRanges left, RowRanges right) { + return INTERSECTION.invoke(left, right); + } + + private static final DynFields.UnboundField BLOCK_ROW_RANGES = DynFields.builder() + .hiddenImpl(ParquetFileReader.class, "blockRowRanges") + .build(); + + @SuppressWarnings("unchecked") + public static void setRanges(ParquetFileReader reader, int rowGroupIndex, RowRanges ranges) { + Map rowGroupToRanges = (Map) BLOCK_ROW_RANGES.get(reader); + rowGroupToRanges.put(rowGroupIndex, ranges); + } +} diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java index c1d8b0ccbbad..ffe362758700 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java @@ -100,6 +100,7 @@ private static class FileIterator implements CloseableIterator { private final long totalValues; private final boolean reuseContainers; private final long[] rowGroupsStartRowPos; + private final ParquetIndexPageFilter pageFilter; private int nextRowGroup = 0; private long nextRowGroupStart = 0; @@ -113,6 +114,7 @@ private static class FileIterator implements CloseableIterator { this.totalValues = conf.totalValues(); this.reuseContainers = conf.reuseContainers(); this.rowGroupsStartRowPos = conf.startRowPositions(); + this.pageFilter = conf.pageFilter(); } @Override @@ -142,6 +144,11 @@ private void advance() { reader.skipNextRowGroup(); } + Long filteredRowCount = null; + if (pageFilter != null) { + filteredRowCount = pageFilter.applyIndex(reader, nextRowGroup); + } + PageReadStore pages; try { pages = reader.readNextRowGroup(); @@ -150,7 +157,7 @@ private void advance() { } long rowPosition = rowGroupsStartRowPos[nextRowGroup]; - nextRowGroupStart += pages.getRowCount(); + nextRowGroupStart += filteredRowCount != null ? filteredRowCount : pages.getRowCount(); nextRowGroup += 1; model.setPageSource(pages, rowPosition); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java index 7bb89a30f8e9..1ab70abb17fc 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java @@ -59,6 +59,7 @@ class ReadConf { private final boolean reuseContainers; private final Integer batchSize; private final long[] startRowPositions; + private final ParquetIndexPageFilter pageFilter; // List of column chunk metadata for each row group private final List> columnChunkMetaDataForRowGroups; @@ -106,6 +107,9 @@ class ReadConf { statsFilter = new ParquetMetricsRowGroupFilter(expectedSchema, filter, caseSensitive); dictFilter = new ParquetDictionaryRowGroupFilter(expectedSchema, filter, caseSensitive); bloomFilter = new ParquetBloomRowGroupFilter(expectedSchema, filter, caseSensitive); + this.pageFilter = new ParquetIndexPageFilter(expectedSchema, fileSchema, filter, caseSensitive); + } else { + this.pageFilter = null; } long computedTotalValues = 0L; @@ -155,6 +159,7 @@ private ReadConf(ReadConf toCopy) { this.vectorizedModel = toCopy.vectorizedModel; this.columnChunkMetaDataForRowGroups = toCopy.columnChunkMetaDataForRowGroups; this.startRowPositions = toCopy.startRowPositions; + this.pageFilter = toCopy.pageFilter; } ParquetFileReader reader() { @@ -206,6 +211,10 @@ long[] startRowPositions() { return startRowPositions; } + public ParquetIndexPageFilter pageFilter() { + return pageFilter; + } + long totalValues() { return totalValues; }