Skip to content

SAMZA-2591: Async Commit [2/3]: Task Commit api changes and async commit#1490

Merged
mynameborat merged 9 commits intoapache:state-backend-async-commitfrom
dxichen:task-commit-lifecycle
May 7, 2021
Merged

SAMZA-2591: Async Commit [2/3]: Task Commit api changes and async commit#1490
mynameborat merged 9 commits intoapache:state-backend-async-commitfrom
dxichen:task-commit-lifecycle

Conversation

@dxichen
Copy link
Member

@dxichen dxichen commented Apr 16, 2021

  • Introduce new state backend APIs for blobstore and kafka changelog
  • Change the task commit lifecycle to separate snapshot, upload and cleanup phases
  • Make the TaskInstance commit upload and cleanup phases nonblocking

@dxichen
Copy link
Member Author

dxichen commented Apr 16, 2021

Please disregard the Checkpoint v2 migration commit since it is part of #1489

@dxichen dxichen force-pushed the task-commit-lifecycle branch from 23fdb14 to b03306b Compare April 16, 2021 21:51
Comment on lines 39 to 41
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[P1] Can you elaborate why job and container model are passed down here? Seems like unused parameters in this PR at the least.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

jobmodel is used for potential forwards compatibility, and container model is used for restores, so I wanted to keep it symmetrical.

Comment on lines +47 to +51
TaskRestoreManager getRestoreManager(JobContext jobContext,
ContainerContext containerContext,
TaskModel taskModel,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[P1] Seems inconsistent with the above. Maybe consider passing the same context as part of the signature of getBackupManager too so that access to JobModel and ContainerModel are all through the context variables which makes access pattern consistent in the code base and helps with evolution.

That said, is the TaskContext available before instantiation so that TaskModel is also accessed through the context?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

will change the backup to context passing

Comment on lines +97 to +101
stateBackendToBackupManager.values()
.forEach(storageBackupManager -> storageBackupManager.init(null));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[P2] Can we use a sentinel checkpoint instead of null?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

will address this as a follow up

CheckpointManager checkpointManager, Config config, ExecutorService backupExecutor,
StorageManagerUtil storageManagerUtil, File durableStoreBaseDir) {
this.taskName = taskName;
this.containerStorageManager = containerStorageManager;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[P1] Can we persist the StorageEngines here instead of getting a handle of ContainerStorageManager?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Unfortunately storageEngines are not created at this point, it is created after init is called, that is the reason we are handling containerStorageManager

}

public void init() {
// Assuming that container storage manager has already started and created to stores
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there a way to enforce or validate this assumption? Refer to the above comment on moving this to constructor. Is it here because the data may not be available during construction?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes the data is not avail during construction

Comment on lines +115 to +126
storageEngines.forEach((storeName, storageEngine) -> {
if (storageEngine.getStoreProperties().isPersistedToDisk() &&
storageEngine.getStoreProperties().isDurableStore()) {
storageEngine.checkpoint(checkpointId);
}
});
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is isDurableStore() equivalent to isLoggedStore()? We used to checkpoint for persisted and logged store. Making sure this is just a rename?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

isDurable is a superset of isLogged, isLogged is specifically for kafka durability but isDurable means it is durable either blob store or kafka changelog

Comment on lines +125 to +141
// for each configured state backend factory, backup the state for all stores in this task.
stateBackendToBackupManager.forEach((stateBackendFactoryName, backupManager) -> {
Map<String, String> snapshotSCMs = backupManager.snapshot(checkpointId);
LOG.debug("Created snapshot for taskName: {}, checkpoint id: {}, state backend: {}. Snapshot SCMs: {}",
taskName, checkpointId, stateBackendFactoryName, snapshotSCMs);
stateBackendToStoreSCMs.put(stateBackendFactoryName, snapshotSCMs);
});
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to backup the state for all stores or the stores within each of the backup factories satisfy the above requirement for checkpointing (persisted & durable)?

Looking at the Kafka implementations, storeChangelogs are the ones that are iterated for fetching the snapshot but the above criteria needs both durable & persisted.

Checking if changelog enabled stores are persisted to disk by default and what the behavior is otherwise?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We need to backup all the stores that are durable (or logged) and write as a checkpoint (to file) all the state that are persisted. Similarly, changelog enabled stores are persisted to disk only if they are persisted (ie non inmem), which should perserve the existing behavior.

@dxichen dxichen force-pushed the task-commit-lifecycle branch from b03306b to 0f7be25 Compare April 29, 2021 04:10
@dxichen dxichen force-pushed the task-commit-lifecycle branch from 3008a98 to cef0f1e Compare April 29, 2021 23:33
@mynameborat mynameborat merged commit c117a68 into apache:state-backend-async-commit May 7, 2021
shekhars-li pushed a commit to shekhars-li/samza that referenced this pull request May 28, 2021
…mit (apache#1490)

Introduce new state backend APIs for blobstore and kafka changelog
Change the task commit lifecycle to separate snapshot, upload and cleanup phases
Make the TaskInstance commit upload and cleanup phases nonblocking
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants

Comments