1
0

[HUDI-2914] Fix remote timeline server config for flink (#4191)

This commit is contained in:
Danny Chan
2021-12-03 08:59:10 +08:00
committed by GitHub
parent 91d2e61433
commit 934fe54cc5
7 changed files with 34 additions and 40 deletions

View File

@@ -401,7 +401,7 @@ public class TestBucketAssigner {
}
private static String getLastCompleteInstant(WriteProfile profile) {
return StreamerUtil.getLastCompletedInstant(profile.getTable().getMetaClient());
return StreamerUtil.getLastCompletedInstant(profile.getMetaClient());
}
private void assertBucketEquals(

View File

@@ -19,9 +19,12 @@
package org.apache.hudi.utils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.util.ViewStorageProperties;
import org.apache.flink.configuration.Configuration;
import org.junit.jupiter.api.Test;
@@ -98,5 +101,13 @@ public class TestStreamerUtil {
long diff = StreamerUtil.instantTimeDiffSeconds(higher, lower);
assertThat(diff, is(75L));
}
@Test
void testDumpRemoteViewStorageConfig() throws IOException {
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
StreamerUtil.createWriteClient(conf);
FileSystemViewStorageConfig storageConfig = ViewStorageProperties.loadFromProperties(conf.getString(FlinkOptions.PATH));
assertThat(storageConfig.getStorageType(), is(FileSystemViewStorageType.REMOTE_FIRST));
}
}