2222import org .apache .amoro .IcebergActions ;
2323import org .apache .amoro .TableFormat ;
2424import org .apache .amoro .TableRuntime ;
25- import org .apache .amoro .config .ConfigHelpers ;
25+ import org .apache .amoro .config .ConfigOption ;
26+ import org .apache .amoro .config .ConfigOptions ;
27+ import org .apache .amoro .config .Configurations ;
28+ import org .apache .amoro .process .ExecuteEngine ;
29+ import org .apache .amoro .process .LocalExecutionEngine ;
2630import org .apache .amoro .process .ProcessFactory ;
2731import org .apache .amoro .process .ProcessTriggerStrategy ;
2832import org .apache .amoro .process .RecoverProcessFailedException ;
2933import org .apache .amoro .process .TableProcess ;
3034import org .apache .amoro .process .TableProcessStore ;
31- import org .apache .amoro .server .process .DefaultTableProcessStore ;
32- import org .apache .amoro .server .process .TableProcessMeta ;
33- import org .apache .amoro .server .process .executor .LocalExecutionEngine ;
34- import org .apache .amoro .server .utils .SnowflakeIdGenerator ;
35+ import org .apache .amoro .shade .guava32 .com .google .common .collect .Lists ;
36+ import org .apache .amoro .shade .guava32 .com .google .common .collect .Maps ;
37+ import org .apache .commons .lang3 .tuple .Pair ;
3538
3639import java .time .Duration ;
37- import java .util .Collections ;
38- import java .util .HashMap ;
39- import java .util .HashSet ;
40+ import java .util .Collection ;
41+ import java .util .List ;
4042import java .util .Map ;
4143import java .util .Optional ;
4244import java .util .Set ;
45+ import java .util .stream .Collectors ;
4346
4447/** Default process factory for Iceberg-related maintenance actions in AMS. */
4548public class IcebergProcessFactory implements ProcessFactory {
4649
4750 public static final String PLUGIN_NAME = "iceberg" ;
51+ public static final ConfigOption <Boolean > SNAPSHOT_EXPIRE_ENABLED =
52+ ConfigOptions .key ("expire-snapshots.enabled" ).booleanType ().defaultValue (true );
4853
49- private final SnowflakeIdGenerator idGenerator = new SnowflakeIdGenerator ();
54+ public static final ConfigOption <Duration > SNAPSHOT_EXPIRE_INTERVAL =
55+ ConfigOptions .key ("expire-snapshot.interval" )
56+ .durationType ()
57+ .defaultValue (Duration .ofHours (1 ));
5058
51- private boolean expireSnapshotsEnabled = true ;
52- private int expireSnapshotsThreadCount = 10 ;
53- private Duration expireSnapshotsInterval = Duration .ofHours (1 );
59+ private ExecuteEngine localEngine ;
60+ private final Map <Action , ProcessTriggerStrategy > actions = Maps .newHashMap ();
61+ private final List <TableFormat > formats =
62+ Lists .newArrayList (TableFormat .ICEBERG , TableFormat .MIXED_ICEBERG , TableFormat .MIXED_HIVE );
63+
64+ @ Override
65+ public void availableExecuteEngines (Collection <ExecuteEngine > allAvailableEngines ) {
66+ for (ExecuteEngine engine : allAvailableEngines ) {
67+ if (engine instanceof LocalExecutionEngine ) {
68+ this .localEngine = engine ;
69+ }
70+ }
71+ }
5472
5573 @ Override
5674 public Map <TableFormat , Set <Action >> supportedActions () {
57- Set <Action > actions = new HashSet <>();
58- actions .add (IcebergActions .EXPIRE_SNAPSHOTS );
59-
60- Map <TableFormat , Set <Action >> supported = new HashMap <>();
61- supported .put (TableFormat .ICEBERG , actions );
62- supported .put (TableFormat .MIXED_ICEBERG , actions );
63- supported .put (TableFormat .MIXED_HIVE , actions );
64- return supported ;
75+ return formats .stream ()
76+ .map (f -> Pair .of (f , actions .keySet ()))
77+ .collect (Collectors .toMap (Pair ::getKey , Pair ::getValue ));
6578 }
6679
6780 @ Override
6881 public ProcessTriggerStrategy triggerStrategy (TableFormat format , Action action ) {
69- if (IcebergActions .EXPIRE_SNAPSHOTS .equals (action )) {
70- return new ProcessTriggerStrategy (
71- expireSnapshotsInterval , false , Math .max (expireSnapshotsThreadCount , 1 ));
72- }
73-
74- return ProcessTriggerStrategy .METADATA_TRIGGER ;
82+ return actions .getOrDefault (action , ProcessTriggerStrategy .METADATA_TRIGGER );
7583 }
7684
7785 @ Override
7886 public Optional <TableProcess > trigger (TableRuntime tableRuntime , Action action ) {
79- if (IcebergActions .EXPIRE_SNAPSHOTS .equals (action )
80- && (!expireSnapshotsEnabled
81- || !tableRuntime .getTableConfiguration ().isExpireSnapshotEnabled ())) {
87+ if (!actions .containsKey (action )) {
8288 return Optional .empty ();
8389 }
8490
85- long processId = idGenerator .generateId ();
86- TableProcessMeta meta =
87- TableProcessMeta .of (
88- processId ,
89- tableRuntime .getTableIdentifier ().getId (),
90- action .getName (),
91- LocalExecutionEngine .ENGINE_NAME ,
92- Collections .emptyMap ());
93-
94- TableProcessStore store = new DefaultTableProcessStore (tableRuntime , meta , action );
95-
9691 if (IcebergActions .EXPIRE_SNAPSHOTS .equals (action )) {
97- return Optional . of ( new SnapshotsExpiringProcess ( tableRuntime , store ) );
92+ return triggerExpireSnapshot ( tableRuntime );
9893 }
99-
10094 return Optional .empty ();
10195 }
10296
10397 @ Override
10498 public TableProcess recover (TableRuntime tableRuntime , TableProcessStore store )
10599 throws RecoverProcessFailedException {
106- if (IcebergActions .EXPIRE_SNAPSHOTS .equals (store .getAction ())) {
107- return new SnapshotsExpiringProcess (tableRuntime , store );
108- }
109-
110100 throw new RecoverProcessFailedException (
111101 "Unsupported action for IcebergProcessFactory: " + store .getAction ());
112102 }
@@ -116,42 +106,20 @@ public void open(Map<String, String> properties) {
116106 if (properties == null || properties .isEmpty ()) {
117107 return ;
118108 }
119-
120- expireSnapshotsEnabled =
121- parseBoolean (properties .get ("expire-snapshots.enabled" ), expireSnapshotsEnabled );
122- expireSnapshotsThreadCount =
123- parseInt (properties .get ("expire-snapshots.thread-count" ), expireSnapshotsThreadCount );
124- expireSnapshotsInterval =
125- parseDuration (properties .get ("expire-snapshots.interval" ), expireSnapshotsInterval );
126- }
127-
128- private boolean parseBoolean (String value , boolean defaultValue ) {
129- if (value == null ) {
130- return defaultValue ;
109+ Configurations configs = Configurations .fromMap (properties );
110+ if (configs .getBoolean (SNAPSHOT_EXPIRE_ENABLED )) {
111+ Duration interval = configs .getDuration (SNAPSHOT_EXPIRE_INTERVAL );
112+ this .actions .put (
113+ IcebergActions .EXPIRE_SNAPSHOTS , ProcessTriggerStrategy .triggerAtFixRate (interval ));
131114 }
132- return Boolean .parseBoolean (value .trim ());
133115 }
134116
135- private int parseInt (String value , int defaultValue ) {
136- if (value == null ) {
137- return defaultValue ;
138- }
139- try {
140- return Integer .parseInt (value .trim ());
141- } catch (NumberFormatException e ) {
142- return defaultValue ;
117+ private Optional <TableProcess > triggerExpireSnapshot (TableRuntime tableRuntime ) {
118+ if (localEngine == null ) {
119+ return Optional .empty ();
143120 }
144- }
145121
146- private Duration parseDuration (String value , Duration defaultValue ) {
147- if (value == null ) {
148- return defaultValue ;
149- }
150- try {
151- return ConfigHelpers .TimeUtils .parseDuration (value );
152- } catch (Exception e ) {
153- return defaultValue ;
154- }
122+ return Optional .of (new SnapshotsExpiringProcess (tableRuntime , localEngine ));
155123 }
156124
157125 @ Override
0 commit comments