wgtmac commented on code in PR #3098:
URL: https://github.com/apache/parquet-java/pull/3098#discussion_r1903279719
##########
parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java:
##########
@@ -505,6 +505,82 @@ public <R> R filter(
}
}
+ public static final class Size implements FilterPredicate, Serializable {
+ public enum Operator {
+ EQ,
+ LT,
+ LTE,
+ GT,
+ GTE
Review Comment:
```suggestion
LE,
GT,
GE
```
IIRC, these are commonly used abbreviations?
##########
parquet-generator/src/main/java/org/apache/parquet/filter2/IncrementallyUpdatedFilterPredicateGenerator.java:
##########
@@ -338,6 +342,30 @@ private void addUdpBegin() throws IOException {
+ "\n");
}
+ private void addSizeCase() throws IOException {
+ add(" @Override\n" + " public IncrementallyUpdatedFilterPredicate
visit(Size size) {\n"
+ + " final ValueInspector delegate = (ValueInspector)
size.filter(\n"
+ + " (onEq) -> visit(FilterApi.eq(SIZE_PSUEDOCOLUMN, onEq)),\n"
Review Comment:
The naming here seems not consistent. If we use `onEq`, perhaps we should
stick to `onLt`.
##########
parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java:
##########
@@ -493,6 +494,39 @@ public <T extends Comparable<T>> Boolean visit(Contains<T>
contains) {
return contains.filter(this, (l, r) -> l || r, (l, r) -> l && r, v ->
BLOCK_MIGHT_MATCH);
}
+ @Override
+ public Boolean visit(Size size) {
+ ColumnChunkMetaData meta =
getColumnChunk(size.getColumn().getColumnPath());
+
+ if (meta == null) {
+ // the column isn't in this file, so fail eq/gt/gte targeting size > 0
+ final boolean blockCannotMatch =
+ size.filter((eq) -> eq > 0, (lt) -> false, (lte) -> false, (gt) ->
gt >= 0, (gte) -> gte > 0);
+ return blockCannotMatch ? BLOCK_CANNOT_MATCH : BLOCK_MIGHT_MATCH;
+ }
+
+ try {
+ // We know the block has at least as many array elements as the
dictionary sizes
+ final Set<?> dict = expandDictionary(meta);
+ if (dict == null) {
+ return BLOCK_MIGHT_MATCH;
+ }
+ int numDistinctValues = dict.size();
+ final boolean blockCannotMatch = size.filter(
+ (eq) -> eq < numDistinctValues,
+ (lt) -> lt <= numDistinctValues,
+ (lte) -> lte < numDistinctValues,
+ (gt) -> false,
+ (gte) -> false);
+
Review Comment:
I don't think we can trust `numDistinctValues` because a row group might
contain only a subset of entries from the dictionary.
##########
parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java:
##########
@@ -505,6 +505,82 @@ public <R> R filter(
}
}
+ public static final class Size implements FilterPredicate, Serializable {
+ public enum Operator {
+ EQ,
+ LT,
+ LTE,
+ GT,
+ GTE
+ }
+
+ private final Column<?> column;
+ private final Operator operator;
+ private final long value;
+
+ Size(Column<?> column, Operator operator, long value) {
+ this.column = column;
+ this.operator = operator;
+ if (value < 0) {
+ throw new IllegalArgumentException("Argument to size() operator cannot
be negative: " + value);
+ }
+ this.value = value;
+ }
+
+ @Override
+ public <R> R accept(Visitor<R> visitor) {
+ return visitor.visit(this);
+ }
+
+ public long getValue() {
+ return value;
+ }
+
+ public Column<?> getColumn() {
+ return column;
+ }
+
+ public <R> R filter(
+ Function<Long, R> onEq,
+ Function<Long, R> onLt,
+ Function<Long, R> onLtEq,
+ Function<Long, R> onGt,
+ Function<Long, R> onGtEq) {
+ if (operator == Operator.EQ) {
+ return onEq.apply(value);
+ } else if (operator == Operator.LT) {
+ return onLt.apply(value);
+ } else if (operator == Operator.LTE) {
+ return onLtEq.apply(value);
+ } else if (operator == Operator.GT) {
+ return onGt.apply(value);
+ } else if (operator == Operator.GTE) {
+ return onGtEq.apply(value);
+ } else {
+ throw new UnsupportedOperationException("Operator " + operator + "
cannot be used with size() filter");
+ }
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ return column.equals(((Size) o).column) && operator == ((Size)
o).operator && value == ((Size) o).value;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(column, operator, value);
+ }
+
+ @Override
+ public String toString() {
+ String name = Size.class.getSimpleName().toLowerCase(Locale.ENGLISH);
Review Comment:
It seems an overkill to use the class name. We can directly use `size` here.
##########
parquet-column/src/main/java/org/apache/parquet/filter2/predicate/SchemaCompatibilityValidator.java:
##########
@@ -204,15 +213,21 @@ private <T extends Comparable<T>> void
validateColumn(Column<T> column, boolean
return;
}
- if (shouldBeRepeated && descriptor.getMaxRepetitionLevel() == 0) {
+ if (isRepeatedColumn && descriptor.getMaxRepetitionLevel() == 0) {
throw new IllegalArgumentException(
"FilterPredicate for column " + path.toDotString() + " requires a
repeated "
+ "schema, but found max repetition level " +
descriptor.getMaxRepetitionLevel());
- } else if (!shouldBeRepeated && descriptor.getMaxRepetitionLevel() > 0) {
+ } else if (!isRepeatedColumn && descriptor.getMaxRepetitionLevel() > 0) {
throw new IllegalArgumentException("FilterPredicates do not currently
support repeated columns. "
+ "Column " + path.toDotString() + " is repeated.");
}
+ if (mustBeRequired &&
descriptor.getPrimitiveType().isRepetition(Type.Repetition.OPTIONAL)) {
+ throw new IllegalArgumentException("FilterPredicate for column " +
path.toDotString()
Review Comment:
If the column is `List<List<Int32>>` and elements are required, we can only
support the size operator on the inner list?
##########
parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java:
##########
@@ -505,6 +505,82 @@ public <R> R filter(
}
}
+ public static final class Size implements FilterPredicate, Serializable {
+ public enum Operator {
Review Comment:
Should we support notEqual for completeness, though not that useful?
##########
parquet-generator/src/main/java/org/apache/parquet/filter2/IncrementallyUpdatedFilterPredicateGenerator.java:
##########
@@ -499,6 +527,10 @@ private void addContainsBegin() throws IOException {
+ " @Override\n"
+ " public <T extends Comparable<T>> ContainsPredicate
visit(Contains<T> contains) {\n"
+ " return contains.filter(this, ContainsAndPredicate::new,
ContainsOrPredicate::new, ContainsPredicate::not);\n"
+ + " }\n" + "\n"
+ + " @Override\n"
+ + " public ContainsPredicate visit(Size size) {\n"
+ + " throw new UnsupportedOperationException(\"Unsupported
predicate \" + size + \" cannot be used with contains()\");\n"
Review Comment:
Is this a design limitation or a future work? If latter, perhaps say that it
is not yet implemented?
##########
parquet-column/src/main/java/org/apache/parquet/filter2/recordlevel/IncrementallyUpdatedFilterPredicate.java:
##########
@@ -223,6 +224,74 @@ public void reset() {
}
}
+ class CountingValueInspector extends ValueInspector {
+ private long observedValueCount;
+ private final ValueInspector delegate;
+ private final Function<Long, Boolean> shouldUpdateDelegate;
+
+ public CountingValueInspector(ValueInspector delegate, Function<Long,
Boolean> shouldUpdateDelegate) {
+ this.observedValueCount = 0;
+ this.delegate = delegate;
+ this.shouldUpdateDelegate = shouldUpdateDelegate;
Review Comment:
Perhaps this is worth a comment?
##########
parquet-column/src/main/java/org/apache/parquet/filter2/predicate/SchemaCompatibilityValidator.java:
##########
@@ -175,14 +183,15 @@ private <T extends Comparable<T>> void
validateColumnFilterPredicate(SetColumnFi
}
private <T extends Comparable<T>> void
validateColumnFilterPredicate(Contains<T> pred) {
- validateColumn(pred.getColumn(), true);
+ validateColumn(pred.getColumn(), true, false);
}
private <T extends Comparable<T>> void validateColumn(Column<T> column) {
- validateColumn(column, false);
+ validateColumn(column, false, false);
}
- private <T extends Comparable<T>> void validateColumn(Column<T> column,
boolean shouldBeRepeated) {
+ private <T extends Comparable<T>> void validateColumn(
+ Column<T> column, boolean isRepeatedColumn, boolean mustBeRequired) {
Review Comment:
Is it precise to rename these parameters as below?
- `isRepeatedColumn` -> expectRepeated
- `mustBeRequired` -> expectRequired
##########
parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java:
##########
@@ -505,6 +505,82 @@ public <R> R filter(
}
}
+ public static final class Size implements FilterPredicate, Serializable {
+ public enum Operator {
+ EQ,
+ LT,
+ LTE,
+ GT,
+ GTE
+ }
+
+ private final Column<?> column;
+ private final Operator operator;
+ private final long value;
+
+ Size(Column<?> column, Operator operator, long value) {
+ this.column = column;
+ this.operator = operator;
+ if (value < 0) {
+ throw new IllegalArgumentException("Argument to size() operator cannot
be negative: " + value);
+ }
+ this.value = value;
+ }
+
+ @Override
+ public <R> R accept(Visitor<R> visitor) {
+ return visitor.visit(this);
+ }
+
+ public long getValue() {
+ return value;
+ }
+
+ public Column<?> getColumn() {
+ return column;
+ }
+
+ public <R> R filter(
+ Function<Long, R> onEq,
+ Function<Long, R> onLt,
+ Function<Long, R> onLtEq,
+ Function<Long, R> onGt,
+ Function<Long, R> onGtEq) {
+ if (operator == Operator.EQ) {
+ return onEq.apply(value);
+ } else if (operator == Operator.LT) {
+ return onLt.apply(value);
+ } else if (operator == Operator.LTE) {
+ return onLtEq.apply(value);
+ } else if (operator == Operator.GT) {
+ return onGt.apply(value);
+ } else if (operator == Operator.GTE) {
+ return onGtEq.apply(value);
+ } else {
+ throw new UnsupportedOperationException("Operator " + operator + "
cannot be used with size() filter");
+ }
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ return column.equals(((Size) o).column) && operator == ((Size)
o).operator && value == ((Size) o).value;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(column, operator, value);
+ }
+
+ @Override
+ public String toString() {
+ String name = Size.class.getSimpleName().toLowerCase(Locale.ENGLISH);
+ return name + "(" + operator.toString().toLowerCase() + " " + value +
")";
Review Comment:
Should we include the column name as well?
##########
parquet-generator/src/main/java/org/apache/parquet/filter2/IncrementallyUpdatedFilterPredicateGenerator.java:
##########
@@ -338,6 +342,30 @@ private void addUdpBegin() throws IOException {
+ "\n");
}
+ private void addSizeCase() throws IOException {
+ add(" @Override\n" + " public IncrementallyUpdatedFilterPredicate
visit(Size size) {\n"
+ + " final ValueInspector delegate = (ValueInspector)
size.filter(\n"
+ + " (onEq) -> visit(FilterApi.eq(SIZE_PSUEDOCOLUMN, onEq)),\n"
+ + " (lt) -> visit(FilterApi.lt(SIZE_PSUEDOCOLUMN, lt)),\n"
+ + " (lte) -> visit(FilterApi.ltEq(SIZE_PSUEDOCOLUMN, lte)),\n"
+ + " (gt) -> visit(FilterApi.gt(SIZE_PSUEDOCOLUMN, gt)),\n"
+ + " (gte) -> visit(FilterApi.gtEq(SIZE_PSUEDOCOLUMN, gte)));\n"
+ + "\n"
+ + " final ValueInspector valueInspector = new
IncrementallyUpdatedFilterPredicate.CountingValueInspector(\n"
+ + " delegate,\n"
+ + " size.filter(\n"
Review Comment:
Is the 2nd parameter an optimization to avoid updating the inspector?
##########
parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java:
##########
@@ -505,6 +505,82 @@ public <R> R filter(
}
}
+ public static final class Size implements FilterPredicate, Serializable {
+ public enum Operator {
+ EQ,
+ LT,
+ LTE,
+ GT,
+ GTE
+ }
+
+ private final Column<?> column;
+ private final Operator operator;
+ private final long value;
Review Comment:
Is `int` sufficient?
##########
parquet-column/src/main/java/org/apache/parquet/filter2/recordlevel/IncrementallyUpdatedFilterPredicateBuilderBase.java:
##########
@@ -56,6 +60,8 @@
*/
public abstract class IncrementallyUpdatedFilterPredicateBuilderBase
implements Visitor<IncrementallyUpdatedFilterPredicate> {
+ static final Operators.LongColumn SIZE_PSUEDOCOLUMN =
FilterApi.longColumn("$SIZE");
Review Comment:
Add a comment to explain its use case?
##########
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##########
@@ -217,6 +219,70 @@ public <T extends Comparable<T>> Boolean visit(Contains<T>
contains) {
return contains.filter(this, (l, r) -> l || r, (l, r) -> l && r, v ->
BLOCK_MIGHT_MATCH);
}
+ @Override
+ public Boolean visit(Size size) {
+ final ColumnChunkMetaData metadata =
getColumnChunk(size.getColumn().getColumnPath());
+ if (metadata == null) {
+ // the column isn't in this file, so fail eq/gt/gte targeting size > 0
+ final boolean blockCannotMatch =
+ size.filter((eq) -> eq > 0, (lt) -> false, (lte) -> false, (gt) ->
gt >= 0, (gte) -> gte > 0);
+ return blockCannotMatch ? BLOCK_CANNOT_MATCH : BLOCK_MIGHT_MATCH;
+ }
+
+ final SizeStatistics stats = metadata.getSizeStatistics();
+ final List<Long> repetitionLevelHistogram =
stats.getRepetitionLevelHistogram();
+ final List<Long> definitionLevelHistogram =
stats.getDefinitionLevelHistogram();
+
+ if (repetitionLevelHistogram.isEmpty() ||
definitionLevelHistogram.isEmpty()) {
+ return BLOCK_MIGHT_MATCH;
+ }
+
+ // If all values have repetition level 0, then no array has more than 1
element
+ if (repetitionLevelHistogram.size() == 1
+ || repetitionLevelHistogram.subList(1,
repetitionLevelHistogram.size()).stream()
+ .allMatch(l -> l == 0)) {
+
+ // Null list fields are treated as having size 0
+ if (( // all lists are nulls
+ definitionLevelHistogram.subList(1,
definitionLevelHistogram.size()).stream()
+ .allMatch(l -> l == 0))
+ || // all lists are size 0
+ (definitionLevelHistogram.get(0) == 0
+ && definitionLevelHistogram.subList(2,
definitionLevelHistogram.size()).stream()
+ .allMatch(l -> l == 0))) {
+
+ final boolean blockCannotMatch =
+ size.filter((eq) -> eq > 0, (lt) -> false, (lte) -> false, (gt) ->
gt >= 0, (gte) -> gte > 0);
+ return blockCannotMatch ? BLOCK_CANNOT_MATCH : BLOCK_MIGHT_MATCH;
+ }
+
+ long maxDefinitionLevel =
definitionLevelHistogram.get(definitionLevelHistogram.size() - 1);
+
+ // If all repetition levels are zero and all definitions level are >
MAX_DEFINITION_LEVEL - 1, all lists
+ // are of size 1
+ if (definitionLevelHistogram.stream().allMatch(l -> l >
maxDefinitionLevel - 1)) {
+ final boolean blockCannotMatch = size.filter(
+ (eq) -> eq != 1, (lt) -> lt <= 1, (lte) -> lte < 1, (gt) -> gt >=
1, (gte) -> gte > 1);
+
+ return blockCannotMatch ? BLOCK_CANNOT_MATCH : BLOCK_MIGHT_MATCH;
+ }
+ }
+ long nonNullElementCount =
+ repetitionLevelHistogram.stream().mapToLong(l -> l).sum() -
definitionLevelHistogram.get(0);
+ long numNonNullRecords = repetitionLevelHistogram.get(0) -
definitionLevelHistogram.get(0);
+
+ // Given the total number of elements and non-null fields, we can compute
the max size of any array field
+ long maxArrayElementCount = 1 + (nonNullElementCount - numNonNullRecords);
+ final boolean blockCannotMatch = size.filter(
+ (eq) -> eq > maxArrayElementCount,
+ (lt) -> false,
+ (lte) -> false,
+ (gt) -> gt >= maxArrayElementCount,
+ (gte) -> gte > maxArrayElementCount);
+
+ return blockCannotMatch ? BLOCK_CANNOT_MATCH : BLOCK_MIGHT_MATCH;
Review Comment:
IIUC, these are true for only un-nested list type, right?
##########
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##########
@@ -217,6 +219,70 @@ public <T extends Comparable<T>> Boolean visit(Contains<T>
contains) {
return contains.filter(this, (l, r) -> l || r, (l, r) -> l && r, v ->
BLOCK_MIGHT_MATCH);
}
+ @Override
+ public Boolean visit(Size size) {
+ final ColumnChunkMetaData metadata =
getColumnChunk(size.getColumn().getColumnPath());
+ if (metadata == null) {
+ // the column isn't in this file, so fail eq/gt/gte targeting size > 0
+ final boolean blockCannotMatch =
+ size.filter((eq) -> eq > 0, (lt) -> false, (lte) -> false, (gt) ->
gt >= 0, (gte) -> gte > 0);
+ return blockCannotMatch ? BLOCK_CANNOT_MATCH : BLOCK_MIGHT_MATCH;
+ }
+
+ final SizeStatistics stats = metadata.getSizeStatistics();
Review Comment:
Should we check whether the list type can be supported before proceeding?
For example, perhaps we don't want to support legacy LIST-annotated 2-level
structure and unannotated repeated field according to
https://github.com/apache/parquet-format/pull/466.
##########
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##########
@@ -217,6 +219,70 @@ public <T extends Comparable<T>> Boolean visit(Contains<T>
contains) {
return contains.filter(this, (l, r) -> l || r, (l, r) -> l && r, v ->
BLOCK_MIGHT_MATCH);
}
+ @Override
+ public Boolean visit(Size size) {
+ final ColumnChunkMetaData metadata =
getColumnChunk(size.getColumn().getColumnPath());
+ if (metadata == null) {
+ // the column isn't in this file, so fail eq/gt/gte targeting size > 0
+ final boolean blockCannotMatch =
+ size.filter((eq) -> eq > 0, (lt) -> false, (lte) -> false, (gt) ->
gt >= 0, (gte) -> gte > 0);
+ return blockCannotMatch ? BLOCK_CANNOT_MATCH : BLOCK_MIGHT_MATCH;
Review Comment:
Shouldn't it always return `BLOCK_CANNOT_MATCH`? What is the value of
`size(null list)`? null or 0?
##########
parquet-column/src/main/java/org/apache/parquet/filter2/recordlevel/IncrementallyUpdatedFilterPredicate.java:
##########
@@ -223,6 +224,74 @@ public void reset() {
}
}
+ class CountingValueInspector extends ValueInspector {
+ private long observedValueCount;
+ private final ValueInspector delegate;
+ private final Function<Long, Boolean> shouldUpdateDelegate;
+
+ public CountingValueInspector(ValueInspector delegate, Function<Long,
Boolean> shouldUpdateDelegate) {
+ this.observedValueCount = 0;
+ this.delegate = delegate;
+ this.shouldUpdateDelegate = shouldUpdateDelegate;
+ }
+
+ @Override
+ public void updateNull() {
+ delegate.update(observedValueCount);
+ if (!delegate.isKnown()) {
+ delegate.updateNull();
+ }
+ setResult(delegate.getResult());
+ }
+
+ @Override
+ public void update(int value) {
+ incrementCount();
+ }
+
+ @Override
+ public void update(long value) {
+ incrementCount();
+ }
+
+ @Override
+ public void update(double value) {
+ incrementCount();
+ }
+
+ @Override
+ public void update(float value) {
+ incrementCount();
+ }
+
+ @Override
+ public void update(boolean value) {
+ incrementCount();
+ }
+
+ @Override
+ public void update(Binary value) {
+ incrementCount();
+ }
+
+ @Override
+ public void reset() {
+ super.reset();
+ delegate.reset();
+ observedValueCount = 0;
+ }
+
+ private void incrementCount() {
+ observedValueCount++;
Review Comment:
Is it better to check whether the result is known before doing anything?
##########
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##########
@@ -217,6 +219,70 @@ public <T extends Comparable<T>> Boolean visit(Contains<T>
contains) {
return contains.filter(this, (l, r) -> l || r, (l, r) -> l && r, v ->
BLOCK_MIGHT_MATCH);
}
+ @Override
+ public Boolean visit(Size size) {
+ final ColumnChunkMetaData metadata =
getColumnChunk(size.getColumn().getColumnPath());
+ if (metadata == null) {
+ // the column isn't in this file, so fail eq/gt/gte targeting size > 0
+ final boolean blockCannotMatch =
+ size.filter((eq) -> eq > 0, (lt) -> false, (lte) -> false, (gt) ->
gt >= 0, (gte) -> gte > 0);
+ return blockCannotMatch ? BLOCK_CANNOT_MATCH : BLOCK_MIGHT_MATCH;
Review Comment:
If the behavior depends on SQL dialect, perhaps we should be conservative to
return `BLOCK_MIGHT_MATCH` instead.
##########
parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/ColumnIndexBuilder.java:
##########
@@ -378,6 +379,11 @@ public <T extends Comparable<T>> PrimitiveIterator.OfInt
visit(Contains<T> conta
indices -> IndexIterator.all(getPageCount()));
}
+ @Override
+ public PrimitiveIterator.OfInt visit(Size size) {
+ return IndexIterator.all(getPageCount());
Review Comment:
Repeating the same logic is necessary if you want to support page-level
filtering.
##########
parquet-generator/src/main/java/org/apache/parquet/filter2/IncrementallyUpdatedFilterPredicateGenerator.java:
##########
@@ -338,6 +342,30 @@ private void addUdpBegin() throws IOException {
+ "\n");
}
+ private void addSizeCase() throws IOException {
+ add(" @Override\n" + " public IncrementallyUpdatedFilterPredicate
visit(Size size) {\n"
+ + " final ValueInspector delegate = (ValueInspector)
size.filter(\n"
+ + " (onEq) -> visit(FilterApi.eq(SIZE_PSUEDOCOLUMN, onEq)),\n"
+ + " (lt) -> visit(FilterApi.lt(SIZE_PSUEDOCOLUMN, lt)),\n"
+ + " (lte) -> visit(FilterApi.ltEq(SIZE_PSUEDOCOLUMN, lte)),\n"
+ + " (gt) -> visit(FilterApi.gt(SIZE_PSUEDOCOLUMN, gt)),\n"
+ + " (gte) -> visit(FilterApi.gtEq(SIZE_PSUEDOCOLUMN, gte)));\n"
+ + "\n"
+ + " final ValueInspector valueInspector = new
IncrementallyUpdatedFilterPredicate.CountingValueInspector(\n"
+ + " delegate,\n"
+ + " size.filter(\n"
Review Comment:
BTW, it seems that we can greatly simplify the logic by checking `count >
value` in all cases.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]