Skip to content

Commit

Permalink
ORC-27: Add support for proleptic Gregorian calendar for better suppo…
Browse files Browse the repository at this point in the history
…rt of

dates before 1600AD.

Fixes #455

Signed-off-by: Owen O'Malley <omalley@apache.org>
  • Loading branch information
omalley committed Dec 4, 2019
1 parent 1713266 commit 949c744
Show file tree
Hide file tree
Showing 29 changed files with 670 additions and 61 deletions.
11 changes: 10 additions & 1 deletion java/core/src/java/org/apache/orc/OrcConf.java
Original file line number Diff line number Diff line change
Expand Up @@ -165,7 +165,16 @@ public enum OrcConf {
Integer.MAX_VALUE - 1024, "When reading stripes >2GB, specify max limit for the chunk size."),
ENCRYPTION("orc.encrypt", "orc.encrypt", null, "The list of keys and columns to encrypt with"),
DATA_MASK("orc.mask", "orc.mask", null, "The masks to apply to the encrypted columns"),
KEY_PROVIDER("orc.key.provider", "orc.key.provider", "hadoop", "The kind of KeyProvider to use for encryption.")
KEY_PROVIDER("orc.key.provider", "orc.key.provider", "hadoop", "The kind of KeyProvider to use for encryption."),
PROLEPTIC_GREGORIAN("orc.proleptic.gregorian", "orc.proleptic.gregorian", false,
"Should we read and write dates & times using the proleptic Gregorian calendar\n" +
"instead of the hybrid Julian Gregorian? Hive before 3.1 and Spark before 3.0\n" +
"used hybrid."),
PROLEPTIC_GREGORIAN_DEFAULT("orc.proleptic.gregorian.default",
"orc.proleptic.gregorian.default", false,
"This value controls whether pre-ORC 27 files are using the hybrid or proleptic\n" +
"calendar. Only Hive 3.1 and the C++ library wrote using the proleptic, so hybrid\n" +
"is the default.")
;

private final String attribute;
Expand Down
40 changes: 39 additions & 1 deletion java/core/src/java/org/apache/orc/OrcFile.java
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,6 @@
import org.apache.orc.impl.ReaderImpl;
import org.apache.orc.impl.WriterImpl;
import org.apache.orc.impl.WriterInternal;
import org.apache.orc.impl.reader.ReaderEncryptionVariant;
import org.apache.orc.impl.writer.WriterImplV2;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -283,9 +282,11 @@ public static class ReaderOptions {
// For now keeping this around to avoid complex surgery
private FileMetadata fileMetadata;
private boolean useUTCTimestamp;
private boolean useProlepticGregorian;

public ReaderOptions(Configuration conf) {
this.conf = conf;
this.useProlepticGregorian = OrcConf.PROLEPTIC_GREGORIAN.getBoolean(conf);
}

public ReaderOptions filesystem(FileSystem fs) {
Expand Down Expand Up @@ -313,6 +314,18 @@ public ReaderOptions setKeyProvider(KeyProvider provider) {
return this;
}

/**
* Should the reader convert dates and times to the proleptic Gregorian
* calendar?
* @param newValue should it use the proleptic Gregorian calendar?
* @return this
*/
public ReaderOptions convertToProlepticGregorian(boolean newValue) {
this.useProlepticGregorian = newValue;
return this;
}


public Configuration getConfiguration() {
return conf;
}
Expand Down Expand Up @@ -354,6 +367,9 @@ public boolean getUseUTCTimestamp() {
return useUTCTimestamp;
}

public boolean getConvertToProlepticGregorian() {
return useProlepticGregorian;
}
}

public static ReaderOptions readerOptions(Configuration conf) {
Expand Down Expand Up @@ -434,6 +450,7 @@ public static class WriterOptions implements Cloneable {
private String encryption;
private String masks;
private KeyProvider provider;
private boolean useProlepticGregorian;
private Map<String, HadoopShims.KeyMetadata> keyOverrides = new HashMap<>();

protected WriterOptions(Properties tableProperties, Configuration conf) {
Expand Down Expand Up @@ -479,6 +496,7 @@ protected WriterOptions(Properties tableProperties, Configuration conf) {
OrcConf.WRITE_VARIABLE_LENGTH_BLOCKS.getBoolean(tableProperties,conf);
directEncodingColumns = OrcConf.DIRECT_ENCODING_COLUMNS.getString(
tableProperties, conf);
useProlepticGregorian = OrcConf.PROLEPTIC_GREGORIAN.getBoolean(conf);
}

/**
Expand Down Expand Up @@ -800,6 +818,17 @@ public WriterOptions setKeyProvider(KeyProvider provider) {
return this;
}

/**
* Should the writer use the proleptic Gregorian calendar for
* times and dates.
* @param newValue true if we should use the proleptic calendar
* @return this
*/
public WriterOptions setProlepticGregorian(boolean newValue) {
this.useProlepticGregorian = newValue;
return this;
}

public KeyProvider getKeyProvider() {
return provider;
}
Expand Down Expand Up @@ -919,6 +948,10 @@ public String getMasks() {
public Map<String, HadoopShims.KeyMetadata> getKeyOverrides() {
return keyOverrides;
}

public boolean getProlepticGregorian() {
return useProlepticGregorian;
}
}

/**
Expand Down Expand Up @@ -1129,6 +1162,11 @@ static boolean readerIsCompatible(Reader firstReader,
LOG.info("Can't merge {} because it has different encryption variants", path);
return false;
}
if (firstReader.writerUsedProlepticGregorian() !=
reader.writerUsedProlepticGregorian()) {
LOG.info("Can't merge {} because it uses a different calendar", path);
return false;
}
return true;
}

Expand Down
5 changes: 5 additions & 0 deletions java/core/src/java/org/apache/orc/Reader.java
Original file line number Diff line number Diff line change
Expand Up @@ -495,4 +495,9 @@ public boolean getTolerateMissingSchema() {
* @return Serialized file metadata read from disk for the purposes of caching, etc.
*/
ByteBuffer getSerializedFileFooter();

/**
* Was the file written using the proleptic Gregorian calendar.
*/
boolean writerUsedProlepticGregorian();
}
15 changes: 11 additions & 4 deletions java/core/src/java/org/apache/orc/StripeStatistics.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.orc;

import org.apache.orc.impl.ColumnStatisticsImpl;
import org.apache.orc.impl.ReaderImpl;

import java.util.List;

Expand All @@ -28,15 +29,19 @@
public class StripeStatistics {
protected final List<OrcProto.ColumnStatistics> cs;
protected final TypeDescription schema;
private final ReaderImpl reader;

public StripeStatistics(List<OrcProto.ColumnStatistics> list) {
this(null, list);
public StripeStatistics(List<OrcProto.ColumnStatistics> list,
ReaderImpl reader) {
this(null, list, reader);
}

public StripeStatistics(TypeDescription schema,
List<OrcProto.ColumnStatistics> list) {
List<OrcProto.ColumnStatistics> list,
ReaderImpl reader) {
this.schema = schema;
this.cs = list;
this.reader = reader;
}

private int getBase() {
Expand All @@ -53,7 +58,9 @@ public ColumnStatistics[] getColumnStatistics() {
int base = getBase();
for (int c = 0; c < result.length; ++c) {
TypeDescription column = schema == null ? null : schema.findSubtype(base + c);
result[c] = ColumnStatisticsImpl.deserialize(column, cs.get(c));
result[c] = reader == null
? ColumnStatisticsImpl.deserialize(column, cs.get(c))
: ColumnStatisticsImpl.deserialize(column, cs.get(c), reader);
}
return result;
}
Expand Down
61 changes: 46 additions & 15 deletions java/core/src/java/org/apache/orc/impl/ColumnStatisticsImpl.java
Original file line number Diff line number Diff line change
Expand Up @@ -1479,15 +1479,19 @@ private static final class DateStatisticsImpl extends ColumnStatisticsImpl
DateStatisticsImpl() {
}

DateStatisticsImpl(OrcProto.ColumnStatistics stats) {
DateStatisticsImpl(OrcProto.ColumnStatistics stats,
boolean writerUsedProlepticGregorian,
boolean convertToProlepticGregorian) {
super(stats);
OrcProto.DateStatistics dateStats = stats.getDateStatistics();
// min,max values serialized/deserialized as int (days since epoch)
if (dateStats.hasMaximum()) {
maximum = dateStats.getMaximum();
maximum = DateUtils.convertDate(dateStats.getMaximum(),
writerUsedProlepticGregorian, convertToProlepticGregorian);
}
if (dateStats.hasMinimum()) {
minimum = dateStats.getMinimum();
minimum = DateUtils.convertDate(dateStats.getMinimum(),
writerUsedProlepticGregorian, convertToProlepticGregorian);
}
}

Expand Down Expand Up @@ -1640,23 +1644,31 @@ private static class TimestampStatisticsImpl extends ColumnStatisticsImpl
TimestampStatisticsImpl() {
}

TimestampStatisticsImpl(OrcProto.ColumnStatistics stats) {
TimestampStatisticsImpl(OrcProto.ColumnStatistics stats,
boolean writerUsedProlepticGregorian,
boolean convertToProlepticGregorian) {
super(stats);
OrcProto.TimestampStatistics timestampStats = stats.getTimestampStatistics();
// min,max values serialized/deserialized as int (milliseconds since epoch)
if (timestampStats.hasMaximum()) {
maximum = SerializationUtils.convertToUtc(TimeZone.getDefault(),
timestampStats.getMaximum());
maximum = DateUtils.convertTime(
SerializationUtils.convertToUtc(TimeZone.getDefault(),
timestampStats.getMaximum()),
writerUsedProlepticGregorian, convertToProlepticGregorian);
}
if (timestampStats.hasMinimum()) {
minimum = SerializationUtils.convertToUtc(TimeZone.getDefault(),
timestampStats.getMinimum());
minimum = DateUtils.convertTime(
SerializationUtils.convertToUtc(TimeZone.getDefault(),
timestampStats.getMinimum()),
writerUsedProlepticGregorian, convertToProlepticGregorian);
}
if (timestampStats.hasMaximumUtc()) {
maximum = timestampStats.getMaximumUtc();
maximum = DateUtils.convertTime(timestampStats.getMaximumUtc(),
writerUsedProlepticGregorian, convertToProlepticGregorian);
}
if (timestampStats.hasMinimumUtc()) {
minimum = timestampStats.getMinimumUtc();
minimum = DateUtils.convertTime(timestampStats.getMinimumUtc(),
writerUsedProlepticGregorian, convertToProlepticGregorian);
}
}

Expand Down Expand Up @@ -1795,8 +1807,10 @@ private static final class TimestampInstantStatisticsImpl extends TimestampStati
TimestampInstantStatisticsImpl() {
}

TimestampInstantStatisticsImpl(OrcProto.ColumnStatistics stats) {
super(stats);
TimestampInstantStatisticsImpl(OrcProto.ColumnStatistics stats,
boolean writerUsedProlepticGregorian,
boolean convertToProlepticGregorian) {
super(stats, writerUsedProlepticGregorian, convertToProlepticGregorian);
}

@Override
Expand Down Expand Up @@ -2008,6 +2022,20 @@ public static ColumnStatisticsImpl create(TypeDescription schema) {

public static ColumnStatisticsImpl deserialize(TypeDescription schema,
OrcProto.ColumnStatistics stats) {
return deserialize(schema, stats, false, false);
}

public static ColumnStatisticsImpl deserialize(TypeDescription schema,
OrcProto.ColumnStatistics stats,
ReaderImpl reader) {
return deserialize(schema, stats, reader.writerUsedProlepticGregorian(),
reader.options.getConvertToProlepticGregorian());
}

public static ColumnStatisticsImpl deserialize(TypeDescription schema,
OrcProto.ColumnStatistics stats,
boolean writerUsedProlepticGregorian,
boolean convertToProlepticGregorian) {
if (stats.hasBucketStatistics()) {
return new BooleanStatisticsImpl(stats);
} else if (stats.hasIntStatistics()) {
Expand All @@ -2026,12 +2054,15 @@ public static ColumnStatisticsImpl deserialize(TypeDescription schema,
return new DecimalStatisticsImpl(stats);
}
} else if (stats.hasDateStatistics()) {
return new DateStatisticsImpl(stats);
return new DateStatisticsImpl(stats, writerUsedProlepticGregorian,
convertToProlepticGregorian);
} else if (stats.hasTimestampStatistics()) {
return schema == null ||
schema.getCategory() == TypeDescription.Category.TIMESTAMP ?
new TimestampStatisticsImpl(stats) :
new TimestampInstantStatisticsImpl(stats);
new TimestampStatisticsImpl(stats,
writerUsedProlepticGregorian, convertToProlepticGregorian) :
new TimestampInstantStatisticsImpl(stats,
writerUsedProlepticGregorian, convertToProlepticGregorian);
} else if(stats.hasBinaryStatistics()) {
return new BinaryStatisticsImpl(stats);
} else {
Expand Down
Loading

0 comments on commit 949c744

Please sign in to comment.