1
0

[HUDI-3191] Rebasing Hive's FileInputFormat onto AbstractHoodieTableFileIndex (#4531)

This commit is contained in:
Alexey Kudinkin
2022-01-18 14:54:51 -08:00
committed by GitHub
parent caeea946fb
commit 4bea758738
21 changed files with 623 additions and 78 deletions

View File

@@ -35,6 +35,8 @@ log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO
log4j.logger.org.apache.parquet=ERROR
log4j.logger.parquet=ERROR
log4j.logger.org.apache.spark=WARN
# Disabling Jetty logs
log4j.logger.org.apache.hudi.org.eclipse.jetty=ERROR
# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support
log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL
log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR

View File

@@ -64,6 +64,8 @@ COPY entrypoint.sh /usr/local/bin/
RUN chmod +x /usr/local/bin/entrypoint.sh
ENV PATH $HIVE_HOME/bin/:$PATH
# NOTE: This is the only battle-proven method to inject jars into Hive CLI
ENV AUX_CLASSPATH=file://${HUDI_HADOOP_BUNDLE}
ENTRYPOINT ["entrypoint.sh"]
CMD startup.sh

View File

@@ -162,7 +162,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
deltaWriteStat.setLogFiles(logFiles);
try {
//save hoodie partition meta in the partition path
// Save hoodie partition meta in the partition path
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, baseInstantTime,
new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath));
partitionMetadata.trySave(getPartitionId());

View File

@@ -35,10 +35,61 @@
<directory>src/main/resources</directory>
</resource>
</resources>
<pluginManagement>
<plugins>
<plugin>
<groupId>net.alchim31.maven</groupId>
<artifactId>scala-maven-plugin</artifactId>
<version>${scala-maven-plugin.version}</version>
<configuration>
<args>
<arg>-nobootcp</arg>
<arg>-target:jvm-1.8</arg>
</args>
<checkMultipleScalaVersions>false</checkMultipleScalaVersions>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
</plugin>
</plugins>
</pluginManagement>
<plugins>
<plugin>
<groupId>org.jacoco</groupId>
<artifactId>jacoco-maven-plugin</artifactId>
<groupId>net.alchim31.maven</groupId>
<artifactId>scala-maven-plugin</artifactId>
<executions>
<execution>
<id>scala-compile-first</id>
<phase>process-resources</phase>
<goals>
<goal>add-source</goal>
<goal>compile</goal>
</goals>
</execution>
<execution>
<id>scala-test-compile</id>
<phase>process-test-resources</phase>
<goals>
<goal>testCompile</goal>
</goals>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<executions>
<execution>
<phase>compile</phase>
<goals>
<goal>compile</goal>
</goals>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
@@ -60,6 +111,14 @@
<groupId>org.apache.rat</groupId>
<artifactId>apache-rat-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.scalastyle</groupId>
<artifactId>scalastyle-maven-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.jacoco</groupId>
<artifactId>jacoco-maven-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
@@ -97,6 +156,13 @@
</build>
<dependencies>
<!-- Scala -->
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
<version>${scala.version}</version>
</dependency>
<!-- Fasterxml -->
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>

View File

@@ -202,7 +202,13 @@ public class FSUtils {
public static String getRelativePartitionPath(Path basePath, Path fullPartitionPath) {
basePath = Path.getPathWithoutSchemeAndAuthority(basePath);
fullPartitionPath = Path.getPathWithoutSchemeAndAuthority(fullPartitionPath);
String fullPartitionPathStr = fullPartitionPath.toString();
if (!fullPartitionPathStr.startsWith(basePath.toString())) {
throw new IllegalArgumentException("Partition path does not belong to base-path");
}
int partitionStartIndex = fullPartitionPathStr.indexOf(basePath.getName(),
basePath.getParent() == null ? 0 : basePath.getParent().toString().length());
// Partition-Path could be empty for non-partitioned tables

View File

@@ -0,0 +1,36 @@
/*
* 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.hudi.common.model;
/**
* Hudi table could be queried in one of the 3 following ways:
*
* <ol>
* <li>Snapshot: snapshot of the table at the given (latest if not provided) instant is queried</li>
* <li>Read Optimized (MOR only): snapshot of the table at the given (latest if not provided)
* instant is queried, but w/o reading any of the delta-log files (only reading base-files)</li>
* <li>Incremental: only records added w/in the given time-window (defined by beginning and ending instant)
* are queried</li>
* </ol>
*/
public enum HoodieTableQueryType {
QUERY_TYPE_SNAPSHOT,
QUERY_TYPE_INCREMENTAL,
QUERY_TYPE_READ_OPTIMIZED
}

View File

@@ -18,17 +18,16 @@
package org.apache.hudi
import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_SNAPSHOT_OPT_VAL}
import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties}
import org.apache.hudi.common.engine.HoodieEngineContext
import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.model.FileSlice
import org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ
import org.apache.hudi.common.model.{FileSlice, HoodieTableQueryType}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.view.{FileSystemViewStorageConfig, HoodieTableFileSystemView}
import scala.collection.JavaConversions._
import scala.collection.JavaConverters._
import scala.collection.JavaConversions._
import scala.collection.mutable
/**
@@ -50,11 +49,14 @@ import scala.collection.mutable
* @param shouldIncludePendingCommits flags whether file-index should exclude any pending operations
* @param fileStatusCache transient cache of fetched [[FileStatus]]es
*/
abstract class AbstractHoodieTableFileIndex(engineContext: HoodieEngineContext,
metaClient: HoodieTableMetaClient,
configProperties: TypedProperties,
specifiedQueryInstant: Option[String] = None,
@transient fileStatusCache: FileStatusCacheTrait) {
abstract class HoodieTableFileIndexBase(engineContext: HoodieEngineContext,
metaClient: HoodieTableMetaClient,
configProperties: TypedProperties,
queryType: HoodieTableQueryType,
protected val queryPaths: Seq[Path],
specifiedQueryInstant: Option[String] = None,
shouldIncludePendingCommits: Boolean = false,
@transient fileStatusCache: FileStatusCacheTrait) {
/**
* Get all completeCommits.
*/
@@ -70,12 +72,11 @@ abstract class AbstractHoodieTableFileIndex(engineContext: HoodieEngineContext,
private lazy val metadataConfig = HoodieMetadataConfig.newBuilder
.fromProperties(configProperties)
.build()
protected val basePath: String = metaClient.getBasePath
private val queryType = configProperties(QUERY_TYPE.key())
private val tableType = metaClient.getTableType
@transient private val queryPath = new Path(configProperties.getOrElse("path", "'path' option required"))
protected val basePath: String = metaClient.getBasePath
@transient
@volatile protected var cachedFileSize: Long = 0L
@transient
@@ -109,12 +110,13 @@ abstract class AbstractHoodieTableFileIndex(engineContext: HoodieEngineContext,
.getOrElse(Array.empty[FileStatus])
metaClient.reloadActiveTimeline()
val activeInstants = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants
val latestInstant = activeInstants.lastInstant()
val activeTimeline = getActiveTimeline
val latestInstant = activeTimeline.lastInstant()
// TODO we can optimize the flow by:
// - First fetch list of files from instants of interest
// - Load FileStatus's
fileSystemView = new HoodieTableFileSystemView(metaClient, activeInstants, allFiles)
fileSystemView = new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles)
val queryInstant = if (specifiedQueryInstant.isDefined) {
specifiedQueryInstant
} else if (latestInstant.isPresent) {
@@ -124,7 +126,7 @@ abstract class AbstractHoodieTableFileIndex(engineContext: HoodieEngineContext,
}
(tableType, queryType) match {
case (MERGE_ON_READ, QUERY_TYPE_SNAPSHOT_OPT_VAL) =>
case (MERGE_ON_READ, HoodieTableQueryType.QUERY_TYPE_SNAPSHOT) =>
// Fetch and store latest base and log files, and their sizes
cachedAllInputFileSlices = partitionFiles.map(p => {
val latestSlices = if (queryInstant.isDefined) {
@@ -168,6 +170,15 @@ abstract class AbstractHoodieTableFileIndex(engineContext: HoodieEngineContext,
refresh0()
}
private def getActiveTimeline = {
val timeline = metaClient.getActiveTimeline.getCommitsTimeline
if (shouldIncludePendingCommits) {
timeline
} else {
timeline.filterCompletedInstants()
}
}
private def fileSliceSize(fileSlice: FileSlice): Long = {
val logFileSize = fileSlice.getLogFiles.iterator().asScala.map(_.getFileSize).filter(_ > 0).sum
if (fileSlice.getBaseFile.isPresent) {
@@ -216,11 +227,11 @@ abstract class AbstractHoodieTableFileIndex(engineContext: HoodieEngineContext,
}
def getAllQueryPartitionPaths: Seq[PartitionPath] = {
val queryPartitionPath = FSUtils.getRelativePartitionPath(new Path(basePath), queryPath)
val queryRelativePartitionPaths = queryPaths.map(FSUtils.getRelativePartitionPath(new Path(basePath), _))
// Load all the partition path from the basePath, and filter by the query partition path.
// TODO load files from the queryPartitionPath directly.
// TODO load files from the queryRelativePartitionPaths directly.
val partitionPaths = FSUtils.getAllPartitionPaths(engineContext, metadataConfig, basePath).asScala
.filter(_.startsWith(queryPartitionPath))
.filter(path => queryRelativePartitionPaths.exists(path.startsWith))
val partitionSchema = _partitionColumns

View File

@@ -190,6 +190,9 @@ public class TestFSUtils extends HoodieCommonTestHarness {
Path basePath = new Path("/test/apache");
Path partitionPath = new Path("/test/apache/hudi/sub");
assertEquals("hudi/sub", FSUtils.getRelativePartitionPath(basePath, partitionPath));
Path nonPartitionPath = new Path("/test/something/else");
assertThrows(IllegalArgumentException.class, () -> FSUtils.getRelativePartitionPath(basePath, nonPartitionPath));
}
@Test

View File

@@ -30,6 +30,13 @@
</properties>
<dependencies>
<!-- Scala -->
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
<version>${scala.version}</version>
</dependency>
<!-- Hudi -->
<dependency>
<groupId>org.apache.hudi</groupId>

View File

@@ -0,0 +1,100 @@
/*
* 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.hudi.hadoop;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.HoodieTableFileIndexBase;
import org.apache.hudi.FileStatusCacheTrait;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieTableQueryType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.Option;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.Function0;
import scala.collection.JavaConverters;
import java.util.List;
/**
* Implementation of {@link HoodieTableFileIndexBase} for Hive-based query engines
*/
public class HiveHoodieTableFileIndex extends HoodieTableFileIndexBase {
public static final Logger LOG = LoggerFactory.getLogger(HiveHoodieTableFileIndex.class);
public HiveHoodieTableFileIndex(HoodieEngineContext engineContext,
HoodieTableMetaClient metaClient,
TypedProperties configProperties,
HoodieTableQueryType queryType,
List<Path> queryPaths,
Option<String> specifiedQueryInstant,
boolean shouldIncludePendingCommits
) {
super(engineContext,
metaClient,
configProperties,
queryType,
JavaConverters.asScalaBufferConverter(queryPaths).asScala(),
toScalaOption(specifiedQueryInstant),
shouldIncludePendingCommits,
new NoopCache());
}
private static scala.Option<String> toScalaOption(Option<String> opt) {
return scala.Option.apply(opt.orElse(null));
}
@Override
public Object[] parsePartitionColumnValues(String[] partitionColumns, String partitionPath) {
// NOTE: Parsing partition path into partition column values isn't required on Hive,
// since Hive does partition pruning in a different way (based on the input-path being
// fetched by the query engine)
return new Object[0];
}
@Override
public void logInfo(Function0<String> lazyStr) {
LOG.info(lazyStr.apply());
}
@Override
public void logWarning(Function0<String> lazyStr) {
LOG.info(lazyStr.apply());
}
static class NoopCache implements FileStatusCacheTrait {
@Override
public scala.Option<FileStatus[]> get(Path path) {
return scala.Option.empty();
}
@Override
public void put(Path path, FileStatus[] leafFiles) {
// no-op
}
@Override
public void invalidate() {
// no-op
}
}
}

View File

@@ -27,18 +27,33 @@ import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapred.FileInputFormat;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieTableQueryType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
import scala.collection.JavaConverters;
import scala.collection.Seq;
import javax.annotation.Nonnull;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
/**
* Base implementation of the Hive's {@link FileInputFormat} allowing for reading of Hudi's
@@ -67,6 +82,27 @@ public abstract class HoodieFileInputFormatBase extends FileInputFormat<NullWrit
this.conf = conf;
}
@Nonnull
private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieLogFile latestLogFile, Stream<HoodieLogFile> logFiles) {
List<HoodieLogFile> sortedLogFiles = logFiles.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList());
try {
RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(latestLogFile.getFileStatus());
rtFileStatus.setDeltaLogFiles(sortedLogFiles);
return rtFileStatus;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
@Nonnull
private static FileStatus getFileStatusUnchecked(Option<HoodieBaseFile> baseFileOpt) {
try {
return HoodieInputFormatUtils.getFileStatus(baseFileOpt.get());
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
@Override
public FileStatus[] listStatus(JobConf job) throws IOException {
// Segregate inputPaths[] to incremental, snapshot and non hoodie paths
@@ -102,18 +138,98 @@ public abstract class HoodieFileInputFormatBase extends FileInputFormat<NullWrit
// process snapshot queries next.
List<Path> snapshotPaths = inputPathHandler.getSnapshotPaths();
if (snapshotPaths.size() > 0) {
returns.addAll(HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, tableMetaClientMap, snapshotPaths, includeLogFilesForSnapShotView()));
returns.addAll(listStatusForSnapshotMode(job, tableMetaClientMap, snapshotPaths));
}
return returns.toArray(new FileStatus[0]);
}
@Nonnull
private List<FileStatus> listStatusForSnapshotMode(JobConf job,
Map<String, HoodieTableMetaClient> tableMetaClientMap,
List<Path> snapshotPaths) throws IOException {
HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(job);
List<FileStatus> targetFiles = new ArrayList<>();
TypedProperties props = new TypedProperties(new Properties());
Map<HoodieTableMetaClient, List<Path>> groupedPaths =
HoodieInputFormatUtils.groupSnapshotPathsByMetaClient(tableMetaClientMap.values(), snapshotPaths);
for (Map.Entry<HoodieTableMetaClient, List<Path>> entry : groupedPaths.entrySet()) {
HoodieTableMetaClient tableMetaClient = entry.getKey();
List<Path> partitionPaths = entry.getValue();
// Hive job might specify a max commit instant up to which table's state
// should be examined. We simply pass it as query's instant to the file-index
Option<String> queryCommitInstant =
HoodieHiveUtils.getMaxCommit(job, tableMetaClient.getTableConfig().getTableName());
boolean shouldIncludePendingCommits =
HoodieHiveUtils.shouldIncludePendingCommits(job, tableMetaClient.getTableConfig().getTableName());
HiveHoodieTableFileIndex fileIndex =
new HiveHoodieTableFileIndex(
engineContext,
tableMetaClient,
props,
HoodieTableQueryType.QUERY_TYPE_SNAPSHOT,
partitionPaths,
queryCommitInstant,
shouldIncludePendingCommits);
Map<String, Seq<FileSlice>> partitionedFileSlices =
JavaConverters.mapAsJavaMapConverter(fileIndex.listFileSlices()).asJava();
targetFiles.addAll(
partitionedFileSlices.values()
.stream()
.flatMap(seq -> JavaConverters.seqAsJavaListConverter(seq).asJava().stream())
.map(fileSlice -> {
Option<HoodieBaseFile> baseFileOpt = fileSlice.getBaseFile();
Option<HoodieLogFile> latestLogFileOpt = fileSlice.getLatestLogFile();
if (baseFileOpt.isPresent()) {
return getFileStatusUnchecked(baseFileOpt);
} else if (includeLogFilesForSnapShotView() && latestLogFileOpt.isPresent()) {
return createRealtimeFileStatusUnchecked(latestLogFileOpt.get(), fileSlice.getLogFiles());
} else {
throw new IllegalStateException("Invalid state: either base-file or log-file should be present");
}
})
.collect(Collectors.toList())
);
}
// TODO cleanup
validate(targetFiles, listStatusForSnapshotModeLegacy(job, tableMetaClientMap, snapshotPaths));
return targetFiles;
}
private void validate(List<FileStatus> targetFiles, List<FileStatus> legacyFileStatuses) {
List<FileStatus> diff = CollectionUtils.diff(targetFiles, legacyFileStatuses);
checkState(diff.isEmpty(), "Should be empty");
}
@Nonnull
private List<FileStatus> listStatusForSnapshotModeLegacy(JobConf job, Map<String, HoodieTableMetaClient> tableMetaClientMap, List<Path> snapshotPaths) throws IOException {
return HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, tableMetaClientMap, snapshotPaths, includeLogFilesForSnapShotView());
}
/**
* Abstracts and exposes {@link FileInputFormat#listStatus(JobConf)} operation to subclasses that
* lists files (returning an array of {@link FileStatus}) corresponding to the input paths specified
* as part of provided {@link JobConf}
*/
protected final FileStatus[] doListStatus(JobConf job) throws IOException {
return super.listStatus(job);
}
/**
* Achieves listStatus functionality for an incrementally queried table. Instead of listing all
* partitions and then filtering based on the commits of interest, this logic first extracts the
* partitions touched by the desired commits and then lists only those partitions.
*/
protected List<FileStatus> listStatusForIncrementalMode(
JobConf job, HoodieTableMetaClient tableMetaClient, List<Path> inputPaths) throws IOException {
protected List<FileStatus> listStatusForIncrementalMode(JobConf job, HoodieTableMetaClient tableMetaClient, List<Path> inputPaths) throws IOException {
String tableName = tableMetaClient.getTableConfig().getTableName();
Job jobContext = Job.getInstance(job);
Option<HoodieTimeline> timeline = HoodieInputFormatUtils.getFilteredCommitsTimeline(jobContext, tableMetaClient);
@@ -133,13 +249,4 @@ public abstract class HoodieFileInputFormatBase extends FileInputFormat<NullWrit
FileStatus[] fileStatuses = doListStatus(job);
return HoodieInputFormatUtils.filterIncrementalFileStatus(jobContext, tableMetaClient, timeline.get(), fileStatuses, commitsToCheck.get());
}
/**
* Abstracts and exposes {@link FileInputFormat#listStatus(JobConf)} operation to subclasses that
* lists files (returning an array of {@link FileStatus}) corresponding to the input paths specified
* as part of provided {@link JobConf}
*/
protected final FileStatus[] doListStatus(JobConf job) throws IOException {
return super.listStatus(job);
}
}

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.hadoop.utils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hadoop.fs.Path;
@@ -76,6 +77,14 @@ public class HoodieHiveUtils {
public static final Pattern HOODIE_CONSUME_MODE_PATTERN_STRING = Pattern.compile("hoodie\\.(.*)\\.consume\\.mode");
public static final String GLOBALLY_CONSISTENT_READ_TIMESTAMP = "last_replication_timestamp";
public static boolean shouldIncludePendingCommits(JobConf job, String tableName) {
return job.getBoolean(String.format(HOODIE_CONSUME_PENDING_COMMITS, tableName), false);
}
public static Option<String> getMaxCommit(JobConf job, String tableName) {
return Option.ofNullable(job.get(String.format(HOODIE_CONSUME_COMMIT, tableName)));
}
public static boolean stopAtCompaction(JobContext job, String tableName) {
String compactionPropName = String.format(HOODIE_STOP_AT_COMPACTION_PATTERN, tableName);
boolean stopAtCompaction = job.getConfiguration().getBoolean(compactionPropName, true);
@@ -149,23 +158,17 @@ public class HoodieHiveUtils {
* (false or notSet, notSet) -> returns completedTimeline unfiltered
*
* validCommit is one which exists in the timeline being checked and vice versa
*
* @param tableName
* @param job
* @param metaClient
* @return
*/
public static HoodieTimeline getTableTimeline(final String tableName, final JobConf job, final HoodieTableMetaClient metaClient) {
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline();
boolean includePendingCommits = job.getBoolean(String.format(HOODIE_CONSUME_PENDING_COMMITS, tableName), false);
String maxCommit = job.get(String.format(HOODIE_CONSUME_COMMIT, tableName));
boolean includePendingCommits = shouldIncludePendingCommits(job, tableName);
Option<String> maxCommit = getMaxCommit(job, tableName);
if (!includePendingCommits && maxCommit == null) {
return timeline.filterCompletedInstants();
}
HoodieTimeline finalizedTimeline = includePendingCommits ? timeline : timeline.filterCompletedInstants();
return !maxCommit.isPresent() ? finalizedTimeline : filterIfInstantExists(tableName, finalizedTimeline, maxCommit.get());
return filterIfInstantExists(tableName, includePendingCommits ? timeline : timeline.filterCompletedInstants(), maxCommit);
}
private static HoodieTimeline filterIfInstantExists(String tableName, HoodieTimeline timeline, String maxCommit) {

View File

@@ -18,10 +18,13 @@
package org.apache.hudi.hadoop.testutils;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.RawLocalFileSystem;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.log.HoodieLogFormat;
@@ -68,8 +71,10 @@ public class InputFormatTestUtil {
throws IOException {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), HoodieTableType.COPY_ON_WRITE,
baseFileFormat);
java.nio.file.Path partitionPath = basePath.resolve(Paths.get("2016", "05", "01"));
Files.createDirectories(partitionPath);
setupPartition(basePath, partitionPath);
return simulateInserts(partitionPath.toFile(), baseFileFormat.getFileExtension(), "fileId1", numberOfFiles,
commitNumber);
}
@@ -79,8 +84,10 @@ public class InputFormatTestUtil {
throws IOException {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), HoodieTableType.COPY_ON_WRITE,
baseFileFormat);
java.nio.file.Path partitionPath = basePath.resolve(Paths.get("2016", "05", finalLevelPartitionName));
Files.createDirectories(partitionPath);
setupPartition(basePath, partitionPath);
return simulateInserts(partitionPath.toFile(), baseFileFormat.getFileExtension(), "fileId1" + finalLevelPartitionName, numberOfFiles,
commitNumber);
}
@@ -175,8 +182,12 @@ public class InputFormatTestUtil {
public static File prepareParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber, HoodieTableType tableType) throws IOException {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), tableType, HoodieFileFormat.PARQUET);
java.nio.file.Path partitionPath = basePath.resolve(Paths.get("2016", "05", "01"));
setupPartition(basePath, partitionPath);
createData(schema, partitionPath, numberOfFiles, numberOfRecords, commitNumber);
return partitionPath.toFile();
}
@@ -188,8 +199,12 @@ public class InputFormatTestUtil {
public static File prepareSimpleParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber, HoodieTableType tableType) throws Exception {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), tableType, HoodieFileFormat.PARQUET);
java.nio.file.Path partitionPath = basePath.resolve(Paths.get("2016", "05", "01"));
setupPartition(basePath, partitionPath);
createSimpleData(schema, partitionPath, numberOfFiles, numberOfRecords, commitNumber);
return partitionPath.toFile();
}
@@ -211,7 +226,10 @@ public class InputFormatTestUtil {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString());
for (int i = 0; i < numberPartitions; i++) {
java.nio.file.Path partitionPath = basePath.resolve(Paths.get(2016 + i + "", "05", "01"));
setupPartition(basePath, partitionPath);
createData(schema, partitionPath, 1, numberOfRecordsPerPartition, commitNumber);
result.add(partitionPath.toFile());
}
return result;
@@ -400,10 +418,27 @@ public class InputFormatTestUtil {
jobConf.addResource(conf);
}
private static void setupPartition(java.nio.file.Path basePath, java.nio.file.Path partitionPath) throws IOException {
Files.createDirectories(partitionPath);
// Create partition metadata to properly setup table's partition
RawLocalFileSystem lfs = new RawLocalFileSystem();
lfs.setConf(HoodieTestUtils.getDefaultHadoopConf());
HoodiePartitionMetadata partitionMetadata =
new HoodiePartitionMetadata(
new LocalFileSystem(lfs),
"0",
new Path(basePath.toAbsolutePath().toString()),
new Path(partitionPath.toAbsolutePath().toString())
);
partitionMetadata.trySave((int) (Math.random() * 1000));
}
public static void setInputPath(JobConf jobConf, String inputPath) {
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
jobConf.set("map.input.dir", inputPath);
}
}

View File

@@ -40,6 +40,7 @@ import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeoutException;
@@ -83,10 +84,7 @@ public abstract class ITTestBase {
protected DockerClient dockerClient;
protected Map<String, Container> runningContainers;
static String[] getHiveConsoleCommand(String rawCommand) {
String jarCommand = "add jar " + HUDI_HADOOP_BUNDLE + ";";
String fullCommand = jarCommand + rawCommand;
static String[] getHiveConsoleCommand(String hiveExpr) {
List<String> cmd = new ArrayList<>();
cmd.add("hive");
cmd.add("--hiveconf");
@@ -94,7 +92,7 @@ public abstract class ITTestBase {
cmd.add("--hiveconf");
cmd.add("hive.stats.autogather=false");
cmd.add("-e");
cmd.add("\"" + fullCommand + "\"");
cmd.add("\"" + hiveExpr + "\"");
return cmd.toArray(new String[0]);
}
@@ -181,13 +179,32 @@ public abstract class ITTestBase {
private TestExecStartResultCallback executeCommandInDocker(
String containerName, String[] command, boolean expectedToSucceed) throws Exception {
return executeCommandInDocker(containerName, command, true, expectedToSucceed);
return executeCommandInDocker(containerName, command, true, expectedToSucceed, Collections.emptyMap());
}
private TestExecStartResultCallback executeCommandInDocker(
String containerName, String[] command, boolean checkIfSucceed, boolean expectedToSucceed) throws Exception {
Container sparkWorkerContainer = runningContainers.get(containerName);
ExecCreateCmd cmd = dockerClient.execCreateCmd(sparkWorkerContainer.getId()).withCmd(command).withAttachStdout(true)
private TestExecStartResultCallback executeCommandInDocker(String containerName,
String[] command,
boolean checkIfSucceed,
boolean expectedToSucceed) throws Exception {
return executeCommandInDocker(containerName, command, checkIfSucceed, expectedToSucceed, Collections.emptyMap());
}
private TestExecStartResultCallback executeCommandInDocker(String containerName,
String[] command,
boolean checkIfSucceed,
boolean expectedToSucceed,
Map<String, String> env) throws Exception {
Container targetContainer = runningContainers.get(containerName);
List<String> dockerEnv = env.entrySet()
.stream()
.map(e -> String.format("%s=%s", e.getKey(), e.getValue()))
.collect(Collectors.toList());
ExecCreateCmd cmd = dockerClient.execCreateCmd(targetContainer.getId())
.withEnv(dockerEnv)
.withCmd(command)
.withAttachStdout(true)
.withAttachStderr(true);
ExecCreateCmdResponse createCmdResponse = cmd.exec();
@@ -255,7 +272,9 @@ public abstract class ITTestBase {
LOG.info("\n#################################################################################################");
String[] hiveCmd = getHiveConsoleCommand(hiveCommand);
TestExecStartResultCallback callback = executeCommandInDocker(HIVESERVER, hiveCmd, true);
Map<String, String> env = Collections.singletonMap("AUX_CLASSPATH", "file://" + HUDI_HADOOP_BUNDLE);
TestExecStartResultCallback callback =
executeCommandInDocker(HIVESERVER, hiveCmd, true, true, env);
return Pair.of(callback.getStdout().toString().trim(), callback.getStderr().toString().trim());
}

View File

@@ -46,6 +46,7 @@
<configuration>
<args>
<arg>-nobootcp</arg>
<arg>-target:jvm-1.8</arg>
</args>
<checkMultipleScalaVersions>false</checkMultipleScalaVersions>
</configuration>

View File

@@ -66,14 +66,13 @@ case class HoodieFileIndex(spark: SparkSession,
metaClient = metaClient,
schemaSpec = schemaSpec,
configProperties = getConfigProperties(spark, options),
queryPaths = Seq(HoodieFileIndex.getQueryPath(options)),
specifiedQueryInstant = options.get(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key).map(HoodieSqlCommonUtils.formatQueryInstant),
fileStatusCache = fileStatusCache
)
with FileIndex {
@transient private val queryPath = new Path(options.getOrElse("path", "'path' option required"))
override def rootPaths: Seq[Path] = queryPath :: Nil
override def rootPaths: Seq[Path] = queryPaths
def enableDataSkipping(): Boolean = {
options.getOrElse(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(),
@@ -282,4 +281,8 @@ object HoodieFileIndex {
properties.putAll(options.asJava)
properties
}
private def getQueryPath(options: Map[String, String]) = {
new Path(options.getOrElse("path", "'path' option required"))
}
}

View File

@@ -18,10 +18,11 @@
package org.apache.hudi
import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.hudi.SparkHoodieTableFileIndex.generateFieldMap
import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_INCREMENTAL_OPT_VAL, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, QUERY_TYPE_SNAPSHOT_OPT_VAL}
import org.apache.hudi.SparkHoodieTableFileIndex.{deduceQueryType, generateFieldMap}
import org.apache.hudi.client.common.HoodieSparkEngineContext
import org.apache.hudi.common.config.TypedProperties
import org.apache.hudi.common.model.FileSlice
import org.apache.hudi.common.model.{FileSlice, HoodieTableQueryType}
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator}
import org.apache.spark.api.java.JavaSparkContext
@@ -35,8 +36,10 @@ import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{StringType, StructField, StructType}
import org.apache.spark.unsafe.types.UTF8String
import scala.collection.JavaConversions._
/**
* Implementation of the [[AbstractHoodieTableFileIndex]] for Spark
* Implementation of the [[HoodieTableFileIndexBase]] for Spark
*
* @param spark spark session
* @param metaClient Hudi table's meta-client
@@ -49,14 +52,17 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
metaClient: HoodieTableMetaClient,
schemaSpec: Option[StructType],
configProperties: TypedProperties,
queryPaths: Seq[Path],
specifiedQueryInstant: Option[String] = None,
@transient fileStatusCache: FileStatusCache = NoopCache)
extends AbstractHoodieTableFileIndex(
extends HoodieTableFileIndexBase(
engineContext = new HoodieSparkEngineContext(new JavaSparkContext(spark.sparkContext)),
metaClient,
configProperties,
queryType = deduceQueryType(configProperties),
queryPaths,
specifiedQueryInstant,
SparkHoodieTableFileIndex.adapt(fileStatusCache)
fileStatusCache = SparkHoodieTableFileIndex.adapt(fileStatusCache)
)
with SparkAdapterSupport
with Logging {
@@ -278,6 +284,15 @@ object SparkHoodieTableFileIndex {
traverse(Right(structType))
}
private def deduceQueryType(configProperties: TypedProperties): HoodieTableQueryType = {
configProperties(QUERY_TYPE.key()) match {
case QUERY_TYPE_SNAPSHOT_OPT_VAL => HoodieTableQueryType.QUERY_TYPE_SNAPSHOT
case QUERY_TYPE_INCREMENTAL_OPT_VAL => HoodieTableQueryType.QUERY_TYPE_INCREMENTAL
case QUERY_TYPE_READ_OPTIMIZED_OPT_VAL => HoodieTableQueryType.QUERY_TYPE_READ_OPTIMIZED
case _ @ qt => throw new IllegalArgumentException(s"query-type ($qt) not supported")
}
}
private def adapt(cache: FileStatusCache): FileStatusCacheTrait = {
new FileStatusCacheTrait {
override def get(path: Path): Option[Array[FileStatus]] = cache.getLeafFiles(path)

View File

@@ -67,11 +67,14 @@
<include>org.apache.hudi:hudi-common</include>
<include>org.apache.hudi:hudi-hadoop-mr</include>
<!-- TODO(HUDI-3239) remove this -->
<include>org.scala-lang:scala-library</include>
<include>org.apache.parquet:parquet-avro</include>
<include>org.apache.avro:avro</include>
<include>com.esotericsoftware:kryo-shaded</include>
<include>org.objenesis:objenesis</include>
<include>com.esotericsoftware:minlog</include>
<include>org.apache.avro:avro</include>
<include>org.apache.hbase:hbase-common</include>
<include>org.apache.hbase:hbase-client</include>
<include>org.apache.hbase:hbase-protocol</include>
@@ -152,16 +155,27 @@
<version>${project.version}</version>
</dependency>
<!-- Scala -->
<!-- TODO(HUDI-3239) remove this dep -->
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
<version>${scala.version}</version>
</dependency>
<!-- Parquet -->
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-avro</artifactId>
<version>${parquet.version}</version>
<scope>compile</scope>
</dependency>
<!-- Avro -->
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>${avro.version}</version>
<scope>compile</scope>
</dependency>
@@ -172,6 +186,7 @@
<scope>compile</scope>
</dependency>
<!-- HBase -->
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-common</artifactId>

View File

@@ -69,6 +69,9 @@
<include>org.apache.hudi:hudi-sync-common</include>
<include>org.apache.hudi:hudi-hive-sync</include>
<!-- TODO(HUDI-3239) remove this -->
<include>org.scala-lang:scala-library</include>
<include>com.beust:jcommander</include>
<include>org.apache.avro:avro</include>
<include>org.apache.parquet:parquet-avro</include>
@@ -118,27 +121,42 @@
<artifactId>hudi-common</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-hadoop-mr-bundle</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-hive-sync</artifactId>
<version>${project.version}</version>
</dependency>
<!-- Scala -->
<!-- TODO(HUDI-3239) remove this dep -->
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
<version>${scala.version}</version>
</dependency>
<!-- Parquet -->
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-avro</artifactId>
<version>${parquet.version}</version>
<scope>compile</scope>
</dependency>
<!-- Avro -->
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>${avro.version}</version>
<scope>compile</scope>
</dependency>
</dependencies>
</project>

View File

@@ -66,15 +66,20 @@
<includes>
<include>org.apache.hudi:hudi-common</include>
<include>org.apache.hudi:hudi-hadoop-mr</include>
<!-- TODO(HUDI-3239) remove this -->
<include>org.scala-lang:scala-library</include>
<include>org.apache.parquet:parquet-avro</include>
<include>org.apache.avro:avro</include>
<include>org.codehaus.jackson:*</include>
<include>com.esotericsoftware:kryo-shaded</include>
<include>org.objenesis:objenesis</include>
<include>com.esotericsoftware:minlog</include>
<include>org.apache.hbase:hbase-client</include>
<include>org.apache.hbase:hbase-common</include>
<include>org.apache.hbase:hbase-client</include>
<include>org.apache.hbase:hbase-protocol</include>
<include>org.apache.hbase:hbase-server</include>
<include>org.apache.hbase:hbase-annotations</include>
<include>org.apache.htrace:htrace-core</include>
<include>com.yammer.metrics:metrics-core</include>
<include>com.google.guava:guava</include>
@@ -83,6 +88,10 @@
</includes>
</artifactSet>
<relocations>
<relocation>
<pattern>org.apache.avro.</pattern>
<shadedPattern>org.apache.hudi.org.apache.avro.</shadedPattern>
</relocation>
<relocation>
<pattern>org.codehaus.jackson.</pattern>
<shadedPattern>org.apache.hudi.org.codehaus.jackson.</shadedPattern>
@@ -120,8 +129,8 @@
<shadedPattern>${presto.bundle.bootstrap.shade.prefix}org.apache.htrace.</shadedPattern>
</relocation>
<relocation>
<pattern>org.apache.hadoop.hbase.</pattern>
<shadedPattern>${presto.bundle.bootstrap.shade.prefix}org.apache.hadoop.hbase.</shadedPattern>
<pattern>org.apache.parquet.avro.</pattern>
<shadedPattern>${presto.bundle.bootstrap.shade.prefix}org.apache.parquet.avro.</shadedPattern>
</relocation>
</relocations>
<createDependencyReducedPom>false</createDependencyReducedPom>
@@ -134,7 +143,6 @@
<exclude>META-INF/*.RSA</exclude>
<exclude>META-INF/services/javax.*</exclude>
<exclude>com/esotericsoftware/reflectasm/**</exclude>
<exclude>avro/shaded/com/google/common/**</exclude>
<exclude>stringBehavior.avsc</exclude>
</excludes>
</filter>
@@ -166,6 +174,78 @@
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-hadoop-mr-bundle</artifactId>
<version>${project.version}</version>
<exclusions>
<exclusion>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-common</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-server</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-client</artifactId>
</exclusion>
</exclusions>
</dependency>
<!-- Scala -->
<!-- TODO(HUDI-3239) remove this dep -->
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
<version>${scala.version}</version>
</dependency>
<!-- Parquet -->
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-avro</artifactId>
<scope>compile</scope>
</dependency>
<!-- Avro -->
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<scope>compile</scope>
</dependency>
<!-- HBase -->
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-common</artifactId>
<version>${hbase.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-server</artifactId>
<version>${hbase.version}</version>
<scope>compile</scope>
<exclusions>
<exclusion>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-common</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>org.codehaus.jackson</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>tomcat</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<!--Guava needs to be shaded because HBase 1.2.3 depends on an earlier guava version i.e 12.0.1 and hits runtime

View File

@@ -68,14 +68,17 @@
<include>org.apache.hudi:hudi-common</include>
<include>org.apache.hudi:hudi-hadoop-mr</include>
<!-- TODO(HUDI-3239) remove this -->
<include>org.scala-lang:scala-library</include>
<include>org.apache.parquet:parquet-avro</include>
<include>org.apache.avro:avro</include>
<include>org.codehaus.jackson:*</include>
<include>com.esotericsoftware:kryo-shaded</include>
<include>org.objenesis:objenesis</include>
<include>com.esotericsoftware:minlog</include>
<include>org.apache.hbase:hbase-client</include>
<include>org.apache.hbase:hbase-common</include>
<include>org.apache.hbase:hbase-client</include>
<include>org.apache.hbase:hbase-protocol</include>
<include>org.apache.hbase:hbase-server</include>
<include>org.apache.hbase:hbase-annotations</include>
@@ -186,21 +189,31 @@
</exclusions>
</dependency>
<!-- https://mvnrepository.com/artifact/org.apache.hbase/hbase-shaded-client -->
<!-- Scala -->
<!-- TODO(HUDI-3239) remove this dep -->
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
<version>${scala.version}</version>
</dependency>
<!-- HBase -->
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-shaded-client</artifactId>
<artifactId>hbase-common</artifactId>
<version>${hbase.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-shaded-server</artifactId>
<artifactId>hbase-server</artifactId>
<version>${hbase.version}</version>
<!-- Unfortunately, HFile is packaged ONLY under hbase-server -->
<scope>compile</scope>
<exclusions>
<exclusion>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-common</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>*</artifactId>
@@ -224,12 +237,15 @@
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-avro</artifactId>
<version>${parquet.version}</version>
<scope>compile</scope>
</dependency>
<!-- Avro -->
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>${avro.version}</version>
<scope>compile</scope>
</dependency>