All Downloads are FREE. Search and download functionalities are using the official Maven repository.

org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView Maven / Gradle / Ivy

/*
 * 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.table.view;

import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
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.table.timeline.dto.BaseFileDTO;
import org.apache.hudi.common.table.timeline.dto.ClusteringOpDTO;
import org.apache.hudi.common.table.timeline.dto.CompactionOpDTO;
import org.apache.hudi.common.table.timeline.dto.DTOUtils;
import org.apache.hudi.common.table.timeline.dto.FileGroupDTO;
import org.apache.hudi.common.table.timeline.dto.FileSliceDTO;
import org.apache.hudi.common.table.timeline.dto.InstantDTO;
import org.apache.hudi.common.table.timeline.dto.TimelineDTO;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.RetryHelper;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieRemoteException;

import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.module.afterburner.AfterburnerModule;
import org.apache.http.Consts;
import org.apache.http.client.fluent.Request;
import org.apache.http.client.fluent.Response;
import org.apache.http.client.utils.URIBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.io.Serializable;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;

/**
 * A proxy for table file-system view which translates local View API calls to REST calls to remote timeline service.
 */
public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView, Serializable {

  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper().registerModule(new AfterburnerModule());

  private static final String SCHEME = "http";
  private static final String BASE_URL = "/v1/hoodie/view";
  public static final String LATEST_PARTITION_SLICES_URL = String.format("%s/%s", BASE_URL, "slices/partition/latest/");
  public static final String LATEST_PARTITION_SLICES_INFLIGHT_URL = String.format("%s/%s", BASE_URL, "slices/partition/latest/inflight/");
  public static final String LATEST_PARTITION_SLICES_STATELESS_URL = String.format("%s/%s", BASE_URL, "slices/partition/latest/stateless/");
  public static final String LATEST_PARTITION_SLICE_URL = String.format("%s/%s", BASE_URL, "slices/file/latest/");
  public static final String LATEST_PARTITION_UNCOMPACTED_SLICES_URL =  String.format("%s/%s", BASE_URL, "slices/uncompacted/partition/latest/");
  public static final String ALL_SLICES_URL = String.format("%s/%s", BASE_URL, "slices/all");
  public static final String LATEST_SLICES_MERGED_BEFORE_ON_INSTANT_URL = String.format("%s/%s", BASE_URL, "slices/merged/beforeoron/latest/");
  public static final String LATEST_SLICES_RANGE_INSTANT_URL = String.format("%s/%s", BASE_URL, "slices/range/latest/");
  public static final String LATEST_SLICES_BEFORE_ON_INSTANT_URL = String.format("%s/%s", BASE_URL, "slices/beforeoron/latest/");
  public static final String ALL_LATEST_SLICES_BEFORE_ON_INSTANT_URL = String.format("%s/%s", BASE_URL, "slices/all/beforeoron/latest/");

  public static final String PENDING_COMPACTION_OPS_URL = String.format("%s/%s", BASE_URL, "compactions/pending/");
  public static final String PENDING_LOG_COMPACTION_OPS_URL = String.format("%s/%s", BASE_URL, "logcompactions/pending/");

  public static final String LATEST_PARTITION_DATA_FILES_URL = String.format("%s/%s", BASE_URL, "datafiles/latest/partition");
  public static final String LATEST_PARTITION_DATA_FILE_URL = String.format("%s/%s", BASE_URL, "datafile/latest/partition");
  public static final String ALL_DATA_FILES_URL = String.format("%s/%s", BASE_URL, "datafiles/all");
  public static final String LATEST_ALL_DATA_FILES_URL = String.format("%s/%s", BASE_URL, "datafiles/all/latest/");
  public static final String LATEST_DATA_FILE_ON_INSTANT_URL = String.format("%s/%s", BASE_URL, "datafile/on/latest/");
  public static final String LATEST_DATA_FILES_RANGE_INSTANT_URL = String.format("%s/%s", BASE_URL, "datafiles/range/latest/");
  public static final String LATEST_DATA_FILES_BEFORE_ON_INSTANT_URL = String.format("%s/%s", BASE_URL, "datafiles/beforeoron/latest/");
  public static final String ALL_LATEST_BASE_FILES_BEFORE_ON_INSTANT_URL = String.format("%s/%s", BASE_URL, "basefiles/all/beforeoron/");

  public static final String ALL_FILEGROUPS_FOR_PARTITION_URL = String.format("%s/%s", BASE_URL, "filegroups/all/partition/");
  public static final String ALL_FILEGROUPS_FOR_PARTITION_STATELESS_URL = String.format("%s/%s", BASE_URL, "filegroups/all/partition/stateless/");
  public static final String ALL_REPLACED_FILEGROUPS_BEFORE_OR_ON_URL = String.format("%s/%s", BASE_URL, "filegroups/replaced/beforeoron/");
  public static final String ALL_REPLACED_FILEGROUPS_BEFORE_URL = String.format("%s/%s", BASE_URL, "filegroups/replaced/before/");
  public static final String ALL_REPLACED_FILEGROUPS_AFTER_OR_ON_URL = String.format("%s/%s", BASE_URL, "filegroups/replaced/afteroron/");
  public static final String ALL_REPLACED_FILEGROUPS_PARTITION_URL = String.format("%s/%s", BASE_URL, "filegroups/replaced/partition/");
  
  public static final String PENDING_CLUSTERING_FILEGROUPS_URL = String.format("%s/%s", BASE_URL, "clustering/pending/");

  public static final String LAST_INSTANT_URL = String.format("%s/%s", BASE_URL, "timeline/instant/last");
  public static final String LAST_INSTANTS_URL = String.format("%s/%s", BASE_URL, "timeline/instants/last");

  public static final String TIMELINE_URL = String.format("%s/%s", BASE_URL, "timeline/instants/all");

  // POST Requests
  public static final String REFRESH_TABLE_URL = String.format("%s/%s", BASE_URL, "refresh/");
  public static final String LOAD_ALL_PARTITIONS_URL = String.format("%s/%s", BASE_URL, "loadallpartitions/");
  public static final String LOAD_PARTITIONS_URL = String.format("%s/%s", BASE_URL, "loadpartitions/");

  public static final String PARTITION_PARAM = "partition";
  public static final String PARTITIONS_PARAM = "partitions";
  public static final String BASEPATH_PARAM = "basepath";
  public static final String INSTANT_PARAM = "instant";
  public static final String MAX_INSTANT_PARAM = "maxinstant";
  public static final String MIN_INSTANT_PARAM = "mininstant";
  public static final String INSTANTS_PARAM = "instants";
  public static final String FILEID_PARAM = "fileid";
  public static final String LAST_INSTANT_TS = "lastinstantts";
  public static final String TIMELINE_HASH = "timelinehash";
  public static final String REFRESH_OFF = "refreshoff";
  public static final String INCLUDE_FILES_IN_PENDING_COMPACTION_PARAM = "includependingcompaction";

  public static final String MULTI_VALUE_SEPARATOR = ",";

  private static final Logger LOG = LoggerFactory.getLogger(RemoteHoodieTableFileSystemView.class);
  private static final TypeReference> FILE_SLICE_DTOS_REFERENCE = new TypeReference>() {};
  private static final TypeReference> FILE_GROUP_DTOS_REFERENCE = new TypeReference>() {};
  private static final TypeReference BOOLEAN_TYPE_REFERENCE = new TypeReference() {};
  private static final TypeReference> COMPACTION_OP_DTOS_REFERENCE = new TypeReference>() {};
  private static final TypeReference> CLUSTERING_OP_DTOS_REFERENCE = new TypeReference>() {};
  private static final TypeReference> INSTANT_DTOS_REFERENCE = new TypeReference>() {};
  private static final TypeReference TIMELINE_DTO_REFERENCE = new TypeReference() {};
  private static final TypeReference> BASE_FILE_DTOS_REFERENCE = new TypeReference>() {};
  private static final TypeReference>> BASE_FILE_MAP_REFERENCE = new TypeReference>>() {};
  private static final TypeReference>> FILE_SLICE_MAP_REFERENCE = new TypeReference>>() {};

  private final String serverHost;
  private final int serverPort;
  private final String basePath;
  private final HoodieTableMetaClient metaClient;
  private HoodieTimeline timeline;
  private final int timeoutMs;

  private boolean closed = false;

  private RetryHelper retryHelper;

  private enum RequestMethod {
    GET, POST
  }

  public RemoteHoodieTableFileSystemView(String server, int port, HoodieTableMetaClient metaClient) {
    this(metaClient, FileSystemViewStorageConfig.newBuilder().withRemoteServerHost(server).withRemoteServerPort(port).build());
  }

  public RemoteHoodieTableFileSystemView(HoodieTableMetaClient metaClient, FileSystemViewStorageConfig viewConf) {
    this.basePath = metaClient.getBasePath().toString();
    this.metaClient = metaClient;
    this.timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
    this.serverHost = viewConf.getRemoteViewServerHost();
    this.serverPort = viewConf.getRemoteViewServerPort();
    this.timeoutMs = viewConf.getRemoteTimelineClientTimeoutSecs() * 1000;
    if (viewConf.isRemoteTimelineClientRetryEnabled()) {
      retryHelper = new RetryHelper(
          viewConf.getRemoteTimelineClientMaxRetryIntervalMs(),
          viewConf.getRemoteTimelineClientMaxRetryNumbers(),
          viewConf.getRemoteTimelineInitialRetryIntervalMs(),
          viewConf.getRemoteTimelineClientRetryExceptions(),
          "Sending request");
    }
  }

  private  T executeRequest(String requestPath, Map queryParameters, TypeReference reference,
                               RequestMethod method) throws IOException {
    ValidationUtils.checkArgument(!closed, "View already closed");

    URIBuilder builder = new URIBuilder().setHost(serverHost).setPort(serverPort).setPath(requestPath).setScheme(SCHEME);
    queryParameters.forEach(builder::addParameter);

    // Adding mandatory parameters - Last instants affecting file-slice
    timeline.lastInstant().ifPresent(instant -> builder.addParameter(LAST_INSTANT_TS, instant.getTimestamp()));
    builder.addParameter(TIMELINE_HASH, timeline.getTimelineHash());

    String url = builder.toString();
    LOG.info("Sending request : ({})", url);
    Response response = retryHelper != null ? retryHelper.start(() -> get(timeoutMs, url, method)) : get(timeoutMs, url, method);
    String content = response.returnContent().asString(Consts.UTF_8);
    return (T) OBJECT_MAPPER.readValue(content, reference);
  }

  private Map getParamsWithPartitionPath(String partitionPath) {
    Map paramsMap = new HashMap<>();
    paramsMap.put(BASEPATH_PARAM, basePath);
    paramsMap.put(PARTITION_PARAM, partitionPath);
    return paramsMap;
  }

  private Map getParams() {
    Map paramsMap = new HashMap<>();
    paramsMap.put(BASEPATH_PARAM, basePath);
    return paramsMap;
  }

  private Map getParams(String paramName, String instant) {
    Map paramsMap = new HashMap<>();
    paramsMap.put(BASEPATH_PARAM, basePath);
    paramsMap.put(paramName, instant);
    return paramsMap;
  }

  private Map getParamsWithAdditionalParam(String partitionPath, String paramName, String paramVal) {
    Map paramsMap = new HashMap<>();
    paramsMap.put(BASEPATH_PARAM, basePath);
    paramsMap.put(PARTITION_PARAM, partitionPath);
    paramsMap.put(paramName, paramVal);
    return paramsMap;
  }

  private Map getParamsWithAdditionalParams(String partitionPath, String[] paramNames,
      String[] paramVals) {
    Map paramsMap = new HashMap<>();
    paramsMap.put(BASEPATH_PARAM, basePath);
    paramsMap.put(PARTITION_PARAM, partitionPath);
    ValidationUtils.checkArgument(paramNames.length == paramVals.length);
    for (int i = 0; i < paramNames.length; i++) {
      paramsMap.put(paramNames[i], paramVals[i]);
    }
    return paramsMap;
  }

  private Stream getLatestBaseFilesFromParams(String requestPath, Map paramsMap) {
    try {
      List dataFiles = executeRequest(requestPath, paramsMap,
          BASE_FILE_DTOS_REFERENCE, RequestMethod.GET);
      return dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile);
    } catch (IOException e) {
      throw new HoodieRemoteException(e);
    }
  }

  @Override
  public Stream getLatestBaseFiles(String partitionPath) {
    Map paramsMap = getParamsWithPartitionPath(partitionPath);
    return getLatestBaseFilesFromParams(LATEST_PARTITION_DATA_FILES_URL, paramsMap);
  }

  @Override
  public Stream getLatestBaseFiles() {
    Map paramsMap = getParams();
    return getLatestBaseFilesFromParams(LATEST_ALL_DATA_FILES_URL, paramsMap);
  }

  @Override
  public Stream getLatestBaseFilesBeforeOrOn(String partitionPath, String maxCommitTime) {
    Map paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxCommitTime);
    return getLatestBaseFilesFromParams(LATEST_DATA_FILES_BEFORE_ON_INSTANT_URL, paramsMap);
  }

  @Override
  public Map> getAllLatestBaseFilesBeforeOrOn(String maxCommitTime) {
    Map paramsMap = new HashMap<>();
    paramsMap.put(BASEPATH_PARAM, basePath);
    paramsMap.put(MAX_INSTANT_PARAM, maxCommitTime);

    try {
      Map> dataFileMap = executeRequest(
          ALL_LATEST_BASE_FILES_BEFORE_ON_INSTANT_URL,
          paramsMap,
          BASE_FILE_MAP_REFERENCE,
          RequestMethod.GET);
      return dataFileMap.entrySet().stream().collect(
          Collectors.toMap(
              Map.Entry::getKey,
              entry -> entry.getValue().stream().map(BaseFileDTO::toHoodieBaseFile)));
    } catch (IOException e) {
      throw new HoodieRemoteException(e);
    }
  }

  @Override
  public Option getBaseFileOn(String partitionPath, String instantTime, String fileId) {
    Map paramsMap = getParamsWithAdditionalParams(partitionPath,
        new String[] {INSTANT_PARAM, FILEID_PARAM}, new String[] {instantTime, fileId});
    return Option.fromJavaOptional(getLatestBaseFilesFromParams(LATEST_DATA_FILE_ON_INSTANT_URL, paramsMap).findFirst());
  }

  @Override
  public Stream getLatestBaseFilesInRange(List commitsToReturn) {
    Map paramsMap = getParams(INSTANTS_PARAM, String.join(MULTI_VALUE_SEPARATOR, commitsToReturn));
    return getLatestBaseFilesFromParams(LATEST_DATA_FILES_RANGE_INSTANT_URL, paramsMap);
  }

  @Override
  public Stream getAllBaseFiles(String partitionPath) {
    Map paramsMap = getParamsWithPartitionPath(partitionPath);
    return getLatestBaseFilesFromParams(ALL_DATA_FILES_URL, paramsMap);
  }

  @Override
  public Option getLatestBaseFile(String partitionPath, String fileId) {
    Map paramsMap = getParamsWithAdditionalParam(partitionPath, FILEID_PARAM, fileId);
    return Option.fromJavaOptional(getLatestBaseFilesFromParams(LATEST_PARTITION_DATA_FILE_URL, paramsMap).findFirst());
  }

  private Stream getLatestFileSlicesStreamFromParams(String requestPath, Map paramsMap) {
    try {
      List dataFiles = executeRequest(requestPath, paramsMap,
          FILE_SLICE_DTOS_REFERENCE, RequestMethod.GET);
      return dataFiles.stream().map(FileSliceDTO::toFileSlice);
    } catch (IOException e) {
      throw new HoodieRemoteException(e);
    }
  }

  @Override
  public Stream getLatestFileSlices(String partitionPath) {
    Map paramsMap = getParamsWithPartitionPath(partitionPath);
    return getLatestFileSlicesStreamFromParams(LATEST_PARTITION_SLICES_URL, paramsMap);
  }

  @Override
  public Stream getLatestFileSlicesIncludingInflight(String partitionPath) {
    Map paramsMap = getParamsWithPartitionPath(partitionPath);
    return getLatestFileSlicesStreamFromParams(LATEST_PARTITION_SLICES_INFLIGHT_URL, paramsMap);
  }

  @Override
  public Stream getLatestFileSlicesStateless(String partitionPath) {
    Map paramsMap = getParamsWithPartitionPath(partitionPath);
    return getLatestFileSlicesStreamFromParams(LATEST_PARTITION_SLICES_STATELESS_URL, paramsMap);
  }

  @Override
  public Option getLatestFileSlice(String partitionPath, String fileId) {
    Map paramsMap = getParamsWithAdditionalParam(partitionPath, FILEID_PARAM, fileId);
    return Option.fromJavaOptional(getLatestFileSlicesStreamFromParams(LATEST_PARTITION_SLICE_URL, paramsMap).findFirst());
  }

  @Override
  public Stream getLatestUnCompactedFileSlices(String partitionPath) {
    Map paramsMap = getParamsWithPartitionPath(partitionPath);
    return getLatestFileSlicesStreamFromParams(LATEST_PARTITION_UNCOMPACTED_SLICES_URL, paramsMap);
  }

  @Override
  public Stream getLatestFileSlicesBeforeOrOn(String partitionPath, String maxCommitTime,
      boolean includeFileSlicesInPendingCompaction) {
    Map paramsMap = getParamsWithAdditionalParams(partitionPath,
        new String[] {MAX_INSTANT_PARAM, INCLUDE_FILES_IN_PENDING_COMPACTION_PARAM},
        new String[] {maxCommitTime, String.valueOf(includeFileSlicesInPendingCompaction)});
    return getLatestFileSlicesStreamFromParams(LATEST_SLICES_BEFORE_ON_INSTANT_URL, paramsMap);
  }

  @Override
  public Map> getAllLatestFileSlicesBeforeOrOn(String maxCommitTime) {
    Map paramsMap = new HashMap<>();
    paramsMap.put(BASEPATH_PARAM, basePath);
    paramsMap.put(MAX_INSTANT_PARAM, maxCommitTime);

    try {
      Map> fileSliceMap = executeRequest(ALL_LATEST_SLICES_BEFORE_ON_INSTANT_URL, paramsMap,
          FILE_SLICE_MAP_REFERENCE, RequestMethod.GET);
      return fileSliceMap.entrySet().stream().collect(
          Collectors.toMap(
              Map.Entry::getKey,
              entry -> entry.getValue().stream().map(FileSliceDTO::toFileSlice)));
    } catch (IOException e) {
      throw new HoodieRemoteException(e);
    }
  }

  @Override
  public Stream getLatestMergedFileSlicesBeforeOrOn(String partitionPath, String maxInstantTime) {
    Map paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxInstantTime);
    return getLatestFileSlicesStreamFromParams(LATEST_SLICES_MERGED_BEFORE_ON_INSTANT_URL, paramsMap);
  }

  @Override
  public Stream getLatestFileSliceInRange(List commitsToReturn) {
    Map paramsMap = getParams(INSTANTS_PARAM, String.join(MULTI_VALUE_SEPARATOR, commitsToReturn));
    return getLatestFileSlicesStreamFromParams(LATEST_SLICES_RANGE_INSTANT_URL, paramsMap);
  }

  @Override
  public Stream getAllFileSlices(String partitionPath) {
    Map paramsMap = getParamsWithPartitionPath(partitionPath);
    return getLatestFileSlicesStreamFromParams(ALL_SLICES_URL, paramsMap);
  }

  private Stream getAllFileGroupsForPartitionFromParams(String requestPath, Map paramsMap) {
    try {
      List fileGroups = executeRequest(requestPath, paramsMap,
          FILE_GROUP_DTOS_REFERENCE, RequestMethod.GET);
      return DTOUtils.fileGroupDTOsToFileGroups(fileGroups, metaClient);
    } catch (IOException e) {
      throw new HoodieRemoteException(e);
    }
  }

  @Override
  public Stream getAllFileGroups(String partitionPath) {
    Map paramsMap = getParamsWithPartitionPath(partitionPath);
    return getAllFileGroupsForPartitionFromParams(ALL_FILEGROUPS_FOR_PARTITION_URL, paramsMap);
  }

  @Override
  public Stream getAllFileGroupsStateless(String partitionPath) {
    Map paramsMap = getParamsWithPartitionPath(partitionPath);
    return getAllFileGroupsForPartitionFromParams(ALL_FILEGROUPS_FOR_PARTITION_STATELESS_URL, paramsMap);
  }

  @Override
  public Stream getReplacedFileGroupsBeforeOrOn(String maxCommitTime, String partitionPath) {
    Map paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxCommitTime);
    return getAllFileGroupsForPartitionFromParams(ALL_REPLACED_FILEGROUPS_BEFORE_OR_ON_URL, paramsMap);
  }

  @Override
  public Stream getReplacedFileGroupsBefore(String maxCommitTime, String partitionPath) {
    Map paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxCommitTime);
    return getAllFileGroupsForPartitionFromParams(ALL_REPLACED_FILEGROUPS_BEFORE_URL, paramsMap);
  }

  @Override
  public Stream getReplacedFileGroupsAfterOrOn(String minCommitTime, String partitionPath) {
    Map paramsMap = getParamsWithAdditionalParam(partitionPath, MIN_INSTANT_PARAM, minCommitTime);
    return getAllFileGroupsForPartitionFromParams(ALL_REPLACED_FILEGROUPS_AFTER_OR_ON_URL, paramsMap);
  }

  @Override
  public Stream getAllReplacedFileGroups(String partitionPath) {
    Map paramsMap = getParamsWithPartitionPath(partitionPath);
    return getAllFileGroupsForPartitionFromParams(ALL_REPLACED_FILEGROUPS_PARTITION_URL, paramsMap);
  }

  public boolean refresh() {
    Map paramsMap = getParams();
    try {
      // refresh the local timeline first.
      this.timeline = metaClient.reloadActiveTimeline().filterCompletedAndCompactionInstants();
      return executeRequest(REFRESH_TABLE_URL, paramsMap, BOOLEAN_TYPE_REFERENCE, RequestMethod.POST);
    } catch (IOException e) {
      throw new HoodieRemoteException(e);
    }
  }

  private void loadPartitions(String requestPath, Map paramsMap) {
    try {
      executeRequest(requestPath, paramsMap, BOOLEAN_TYPE_REFERENCE, RequestMethod.POST);
    } catch (IOException e) {
      throw new HoodieRemoteException(e);
    }
  }

  @Override
  public void loadAllPartitions() {
    Map paramsMap = getParams();
    loadPartitions(LOAD_ALL_PARTITIONS_URL, paramsMap);
  }

  @Override
  public void loadPartitions(List partitionPaths) {
    Map paramsMap = getParams();
    try {
      paramsMap.put(PARTITIONS_PARAM, OBJECT_MAPPER.writeValueAsString(partitionPaths));
    } catch (JsonProcessingException e) {
      throw new HoodieRemoteException(e);
    }
    loadPartitions(LOAD_PARTITIONS_URL, paramsMap);
  }

  private Stream> getPendingCompactionOperations(String requestPath, Map paramsMap) {
    try {
      List dtos = executeRequest(requestPath, paramsMap,
          COMPACTION_OP_DTOS_REFERENCE, RequestMethod.GET);
      return dtos.stream().map(CompactionOpDTO::toCompactionOperation);
    } catch (IOException e) {
      throw new HoodieRemoteException(e);
    }
  }

  @Override
  public Stream> getPendingCompactionOperations() {
    Map paramsMap = getParams();
    return getPendingCompactionOperations(PENDING_COMPACTION_OPS_URL, paramsMap);
  }

  @Override
  public Stream> getPendingLogCompactionOperations() {
    Map paramsMap = getParams();
    return getPendingCompactionOperations(PENDING_LOG_COMPACTION_OPS_URL, paramsMap);
  }

  @Override
  public Stream> getFileGroupsInPendingClustering() {
    Map paramsMap = getParams();
    try {
      List dtos = executeRequest(PENDING_CLUSTERING_FILEGROUPS_URL, paramsMap,
          CLUSTERING_OP_DTOS_REFERENCE, RequestMethod.GET);
      return dtos.stream().map(ClusteringOpDTO::toClusteringOperation);
    } catch (IOException e) {
      throw new HoodieRemoteException(e);
    }
  }

  @Override
  public Option getLastInstant() {
    Map paramsMap = getParams();
    try {
      List instants = executeRequest(LAST_INSTANT_URL, paramsMap, INSTANT_DTOS_REFERENCE, RequestMethod.GET);
      return Option.fromJavaOptional(instants.stream().map(InstantDTO::toInstant).findFirst());
    } catch (IOException e) {
      throw new HoodieRemoteException(e);
    }
  }

  @Override
  public HoodieTimeline getTimeline() {
    Map paramsMap = getParams();
    try {
      TimelineDTO timelineDto = executeRequest(TIMELINE_URL, paramsMap, TIMELINE_DTO_REFERENCE, RequestMethod.GET);
      return TimelineDTO.toTimeline(timelineDto, metaClient);
    } catch (IOException e) {
      throw new HoodieRemoteException(e);
    }
  }

  @Override
  public void close() {
    closed = true;
  }

  @Override
  public void reset() {
    refresh();
  }

  @Override
  public void sync() {
    refresh();
  }

  private Response get(int timeoutMs, String url, RequestMethod method) throws IOException {
    switch (method) {
      case GET:
        return Request.Get(url).connectTimeout(timeoutMs).socketTimeout(timeoutMs).execute();
      case POST:
      default:
        return Request.Post(url).connectTimeout(timeoutMs).socketTimeout(timeoutMs).execute();
    }
  }
}




© 2015 - 2024 Weber Informatics LLC | Privacy Policy