-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-4148] Preparations and client for hudi table manager service #5681
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Large diffs are not rendered by default.
Large diffs are not rendered by default.
Large diffs are not rendered by default.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,191 @@ | ||
| /* | ||
| * 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.client.table.manager; | ||
|
|
||
| import org.apache.hudi.common.config.HoodieTableManagerConfig; | ||
| import org.apache.hudi.common.table.HoodieTableMetaClient; | ||
| import org.apache.hudi.common.table.timeline.HoodieInstant; | ||
| import org.apache.hudi.common.util.ClusteringUtils; | ||
| import org.apache.hudi.common.util.StringUtils; | ||
| import org.apache.hudi.common.util.ValidationUtils; | ||
| import org.apache.hudi.exception.HoodieRemoteException; | ||
|
|
||
| import org.apache.http.client.fluent.Request; | ||
| import org.apache.http.client.fluent.Response; | ||
| import org.apache.http.client.utils.URIBuilder; | ||
| import org.apache.logging.log4j.LogManager; | ||
| import org.apache.logging.log4j.Logger; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.HashMap; | ||
| import java.util.Map; | ||
| import java.util.concurrent.TimeUnit; | ||
|
|
||
| /** | ||
| * Client which send the table service instants to the table management service. | ||
| */ | ||
| public class HoodieTableManagerClient { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Handcoding a HTTP Rest client seems very painful / error prone. Why dont we standardize and pick a framework to support any rest endpoints. Jersey is good one. Lets you enable REST endpoints with annotations, provides exception recovery hooks, testability etc.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ditto.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We already use javalin and the rest end points generated/wired that way. Can we just reuse that. it will save all the new bundling work needed for a new dependency. @prasannarajaperumal wdyt?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. javalin seems okay to me. We should use that here.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Need to implement the common HoodieTableServiceClient - Referring to my comment above. |
||
|
|
||
| private static final String BASE_URL = "/v1/hoodie/service"; | ||
|
|
||
| public static final String REGISTER_ENDPOINT = String.format("%s/%s", BASE_URL, "register"); | ||
|
|
||
| public static final String EXECUTE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/execute"); | ||
| public static final String DELETE_COMPACTION = String.format("%s/%s", BASE_URL, "compact/delete"); | ||
|
|
||
| public static final String EXECUTE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/execute"); | ||
| public static final String DELETE_CLUSTERING = String.format("%s/%s", BASE_URL, "cluster/delete"); | ||
|
|
||
| public static final String EXECUTE_CLEAN = String.format("%s/%s", BASE_URL, "clean/execute"); | ||
| public static final String DELETE_CLEAN = String.format("%s/%s", BASE_URL, "clean/delete"); | ||
|
|
||
| public static final String DATABASE_NAME_PARAM = "db_name"; | ||
| public static final String TABLE_NAME_PARAM = "table_name"; | ||
| public static final String BASEPATH_PARAM = "basepath"; | ||
| public static final String INSTANT_PARAM = "instant"; | ||
| public static final String USERNAME = "username"; | ||
| public static final String CLUSTER = "cluster"; | ||
| public static final String QUEUE = "queue"; | ||
| public static final String RESOURCE = "resource"; | ||
| public static final String PARALLELISM = "parallelism"; | ||
| public static final String EXTRA_PARAMS = "extra_params"; | ||
| public static final String EXECUTION_ENGINE = "execution_engine"; | ||
|
|
||
| private final HoodieTableManagerConfig config; | ||
| private final HoodieTableMetaClient metaClient; | ||
| private final String host; | ||
| private final int port; | ||
| private final String basePath; | ||
| private final String dbName; | ||
| private final String tableName; | ||
|
|
||
| private static final Logger LOG = LogManager.getLogger(HoodieTableManagerClient.class); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. probably need to rebase this to log4j2 now
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. will update it.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We can do this in another MR.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. lets actually fix this in this PR please, now that log4j2 is landed.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. fixed. |
||
|
|
||
| public HoodieTableManagerClient(HoodieTableMetaClient metaClient, HoodieTableManagerConfig config) { | ||
| this.basePath = metaClient.getBasePathV2().toString(); | ||
| this.dbName = metaClient.getTableConfig().getDatabaseName(); | ||
| this.tableName = metaClient.getTableConfig().getTableName(); | ||
| this.host = config.getTableManagerHost(); | ||
| this.port = config.getTableManagerPort(); | ||
| this.config = config; | ||
| this.metaClient = metaClient; | ||
| } | ||
|
|
||
| private String executeRequest(String requestPath, Map<String, String> queryParameters) throws IOException { | ||
| URIBuilder builder = | ||
| new URIBuilder().setHost(host).setPort(port).setPath(requestPath).setScheme("http"); | ||
| queryParameters.forEach(builder::addParameter); | ||
|
|
||
| String url = builder.toString(); | ||
| LOG.info("Sending request to table management service : (" + url + ")"); | ||
| Response response; | ||
| int timeout = this.config.getConnectionTimeout() * 1000; // msec | ||
| int requestRetryLimit = config.getConnectionRetryLimit(); | ||
| int retry = 0; | ||
|
|
||
| while (retry < requestRetryLimit) { | ||
| try { | ||
| response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute(); | ||
| return response.returnContent().asString(); | ||
| } catch (IOException e) { | ||
| retry++; | ||
| LOG.warn(String.format("Failed request to server %s, will retry for %d times", url, requestRetryLimit - retry), e); | ||
| if (requestRetryLimit == retry) { | ||
| throw e; | ||
| } | ||
| } | ||
|
|
||
| try { | ||
| TimeUnit.SECONDS.sleep(config.getConnectionRetryDelay()); | ||
| } catch (InterruptedException e) { | ||
| // ignore | ||
| } | ||
| } | ||
|
|
||
| throw new IOException(String.format("Failed request to table management service %s after retry %d times", url, requestRetryLimit)); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. maybe it's better to put this exception on line 110
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. sure, will fixed. |
||
| } | ||
|
|
||
| private Map<String, String> getParamsWithAdditionalParams(String[] paramNames, String[] paramVals) { | ||
| Map<String, String> paramsMap = new HashMap<>(); | ||
| paramsMap.put(BASEPATH_PARAM, basePath); | ||
| ValidationUtils.checkArgument(paramNames.length == paramVals.length); | ||
| for (int i = 0; i < paramNames.length; i++) { | ||
| paramsMap.put(paramNames[i], paramVals[i]); | ||
| } | ||
| return paramsMap; | ||
| } | ||
|
|
||
| public void register() { | ||
| try { | ||
| executeRequest(REGISTER_ENDPOINT, getDefaultParams(null)); | ||
| } catch (IOException e) { | ||
| throw new HoodieRemoteException(e); | ||
| } | ||
| } | ||
|
|
||
| public void executeCompaction() { | ||
| try { | ||
| String instantRange = StringUtils.join(metaClient.reloadActiveTimeline() | ||
| .filterPendingCompactionTimeline() | ||
| .getInstants() | ||
| .map(HoodieInstant::getTimestamp) | ||
| .toArray(String[]::new), ","); | ||
|
|
||
| executeRequest(EXECUTE_COMPACTION, getDefaultParams(instantRange)); | ||
| } catch (IOException e) { | ||
| throw new HoodieRemoteException(e); | ||
| } | ||
| } | ||
|
|
||
| public void executeClean() { | ||
| try { | ||
| String instantRange = StringUtils.join(metaClient.reloadActiveTimeline() | ||
| .getCleanerTimeline() | ||
| .filterInflightsAndRequested() | ||
| .getInstants() | ||
| .map(HoodieInstant::getTimestamp) | ||
| .toArray(String[]::new), ","); | ||
|
|
||
| executeRequest(EXECUTE_CLEAN, getDefaultParams(instantRange)); | ||
| } catch (IOException e) { | ||
| throw new HoodieRemoteException(e); | ||
| } | ||
| } | ||
|
|
||
| public void executeClustering() { | ||
| try { | ||
| metaClient.reloadActiveTimeline(); | ||
| String instantRange = StringUtils.join(ClusteringUtils.getPendingClusteringInstantTimes(metaClient) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I noticed that whenever an action is scheduled, instead of sending that particular instant, we are sending all the pending instants for that action as part of the request. This would mean the remote client has to iterate over the instants and update the contents to the mysql/hudi db. I am guessing this was done so that in case an action is scheduled but failed to update the table service, then the .requested instant will be left on the timeline. One way we are trying to workaround this is by sending the request before creating .requested files. Timeline of actions will look like this,
In case the request is successfully send to table service and the .requested file creation failed then it will be treated as a no-op by table service when it schedules the job. There is a possibility that table service can execute right after point 2, but it can be handled by introducing some delay. This above approach is useful, if we want to send any spark job specific parameters to the table service for executing that instants, example no. of executors. Since, no.of executors can vary depending on the plan.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This will cause two problems:
Considering these two points, I think sending all the pending instant is a better way. WDYT
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Here is my understanding, we have 3 components, scheduler(Hudi table via write API), storage (state manager or metaserver(can be mysql or hudi table)) and execution(Table services).
Another approach, I can think of is to have async thread running that starts on write path and incrementally post the requests to table service reading through the timeline. We can even update file's extrametadata map with any information if needed. I am inclined with your approach because it is a clean way to do it, but we need to address following things,
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. +1 on keeping the table manager/service decouple as discussed.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
+1 from me. |
||
| .stream() | ||
| .map(HoodieInstant::getTimestamp) | ||
| .toArray(String[]::new), ","); | ||
|
|
||
| executeRequest(EXECUTE_CLUSTERING, getDefaultParams(instantRange)); | ||
| } catch (IOException e) { | ||
| throw new HoodieRemoteException(e); | ||
| } | ||
| } | ||
|
|
||
| private Map<String, String> getDefaultParams(String instantRange) { | ||
| return getParamsWithAdditionalParams( | ||
| new String[] {DATABASE_NAME_PARAM, TABLE_NAME_PARAM, INSTANT_PARAM, USERNAME, QUEUE, RESOURCE, PARALLELISM, EXTRA_PARAMS, EXECUTION_ENGINE}, | ||
| new String[] {dbName, tableName, instantRange, config.getDeployUsername(), config.getDeployQueue(), config.getDeployResource(), | ||
| String.valueOf(config.getDeployParallelism()), config.getDeployExtraParams(), config.getDeployExecutionEngine()}); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,6 +29,7 @@ | |
| import org.apache.hudi.common.config.HoodieConfig; | ||
| import org.apache.hudi.common.config.HoodieMetadataConfig; | ||
| import org.apache.hudi.common.config.HoodieMetastoreConfig; | ||
| import org.apache.hudi.common.config.HoodieTableManagerConfig; | ||
| import org.apache.hudi.common.config.TypedProperties; | ||
| import org.apache.hudi.common.engine.EngineType; | ||
| import org.apache.hudi.common.fs.ConsistencyGuardConfig; | ||
|
|
@@ -497,6 +498,7 @@ public class HoodieWriteConfig extends HoodieConfig { | |
| private HoodiePayloadConfig hoodiePayloadConfig; | ||
| private HoodieMetadataConfig metadataConfig; | ||
| private HoodieMetastoreConfig metastoreConfig; | ||
| private HoodieTableManagerConfig tableManagerConfig; | ||
| private HoodieCommonConfig commonConfig; | ||
| private EngineType engineType; | ||
|
|
||
|
|
@@ -889,6 +891,7 @@ protected HoodieWriteConfig(EngineType engineType, Properties props) { | |
| this.hoodiePayloadConfig = HoodiePayloadConfig.newBuilder().fromProperties(newProps).build(); | ||
| this.metadataConfig = HoodieMetadataConfig.newBuilder().fromProperties(props).build(); | ||
| this.metastoreConfig = HoodieMetastoreConfig.newBuilder().fromProperties(props).build(); | ||
| this.tableManagerConfig = HoodieTableManagerConfig.newBuilder().fromProperties(props).build(); | ||
| this.commonConfig = HoodieCommonConfig.newBuilder().fromProperties(props).build(); | ||
| } | ||
|
|
||
|
|
@@ -1948,6 +1951,10 @@ public HoodieMetadataConfig getMetadataConfig() { | |
| return metadataConfig; | ||
| } | ||
|
|
||
| public HoodieTableManagerConfig getTableManagerConfig() { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should we have this as a separate module? That is build on top of
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think TableManagerConfig is a public config, clients can get it through HoodieWriteConfig, so I think the config in HoodieWriteConfig is better. |
||
| return tableManagerConfig; | ||
| } | ||
|
|
||
| public HoodieCommonConfig getCommonConfig() { | ||
| return commonConfig; | ||
| } | ||
|
|
@@ -2160,6 +2167,13 @@ public boolean isMetastoreEnabled() { | |
| return metastoreConfig.enableMetastore(); | ||
| } | ||
|
|
||
| /** | ||
| * Table Manager configs. | ||
| */ | ||
| public boolean isTableManagerEnabled() { | ||
| return tableManagerConfig.enableTableManager(); | ||
| } | ||
|
|
||
| public static class Builder { | ||
|
|
||
| protected final HoodieWriteConfig writeConfig = new HoodieWriteConfig(); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we just align this with how the timeline server is built i.e via javalin instead of the raw HTTP client
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Table Manager Service use the javalin, the implemention of HoodieTableManagerClient align with RemoteHoodieTableFileSystemView.