You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu> on 2021/09/10 22:40:46 UTC

Change in asterixdb[master]: [NO ISSUE][STO] Ensure resources file operations are synchronized

From Murtadha Hubail <mh...@apache.org>:

Murtadha Hubail has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164 )


Change subject: [NO ISSUE][STO] Ensure resources file operations are synchronized
......................................................................

[NO ISSUE][STO] Ensure resources file operations are synchronized

- user model changes: no
- storage format changes: no
- interface changes: no

Details:

- To avoid an operation reading a partially written resource file,
  ensure all such operations are synchronized.

Change-Id: I95f8565780675798393d7d43c0051c04e2b0a98c
---
M asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
1 file changed, 14 insertions(+), 14 deletions(-)



  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/64/13164/1

diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 4d15385..06e4015 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -304,7 +304,7 @@
     }
 
     @Override
-    public long maxId() throws HyracksDataException {
+    public synchronized long maxId() throws HyracksDataException {
         final Map<Long, LocalResource> allResources = loadAndGetAllResources();
         final Optional<Long> max = allResources.keySet().stream().max(Long::compare);
         return max.isPresent() ? max.get() : 0;
@@ -330,7 +330,7 @@
         }
     }
 
-    public void setReplicationManager(IReplicationManager replicationManager) {
+    public synchronized void setReplicationManager(IReplicationManager replicationManager) {
         this.replicationManager = replicationManager;
         isReplicationEnabled = replicationManager.isReplicationEnabled();
 
@@ -357,7 +357,7 @@
      *
      * @throws IOException
      */
-    public void deleteStorageData() throws IOException {
+    public synchronized void deleteStorageData() throws IOException {
         for (Path root : storageRoots) {
             final File rootFile = root.toFile();
             if (rootFile.exists()) {
@@ -367,13 +367,13 @@
         createStorageRoots();
     }
 
-    public Set<Integer> getAllPartitions() throws HyracksDataException {
+    public synchronized Set<Integer> getAllPartitions() throws HyracksDataException {
         return loadAndGetAllResources().values().stream().map(LocalResource::getResource)
                 .map(DatasetLocalResource.class::cast).map(DatasetLocalResource::getPartition)
                 .collect(Collectors.toSet());
     }
 
-    public Optional<DatasetResourceReference> getLocalResourceReference(String absoluteFilePath)
+    public synchronized Optional<DatasetResourceReference> getLocalResourceReference(String absoluteFilePath)
             throws HyracksDataException {
         final String localResourcePath = StoragePathUtil.getIndexFileRelativePath(absoluteFilePath);
         final LocalResource lr = get(localResourcePath);
@@ -388,7 +388,7 @@
      * @return The set of indexes files
      * @throws HyracksDataException
      */
-    public Set<File> getPartitionIndexes(int partition) throws HyracksDataException {
+    public synchronized Set<File> getPartitionIndexes(int partition) throws HyracksDataException {
         final Map<Long, LocalResource> partitionResourcesMap = getResources(resource -> {
             DatasetLocalResource dsResource = (DatasetLocalResource) resource.getResource();
             return dsResource.getPartition() == partition;
@@ -400,14 +400,14 @@
         return indexes;
     }
 
-    public Map<Long, LocalResource> getPartitionResources(int partition) throws HyracksDataException {
+    public synchronized Map<Long, LocalResource> getPartitionResources(int partition) throws HyracksDataException {
         return getResources(resource -> {
             DatasetLocalResource dsResource = (DatasetLocalResource) resource.getResource();
             return dsResource.getPartition() == partition;
         });
     }
 
-    public Map<String, Long> getPartitionReplicatedResources(int partition, IReplicationStrategy strategy)
+    public synchronized Map<String, Long> getPartitionReplicatedResources(int partition, IReplicationStrategy strategy)
             throws HyracksDataException {
         final Map<String, Long> partitionReplicatedResources = new HashMap<>();
         final Map<Long, LocalResource> partitionResources = getPartitionResources(partition);
@@ -421,7 +421,7 @@
         return partitionReplicatedResources;
     }
 
-    public List<String> getPartitionReplicatedFiles(int partition, IReplicationStrategy strategy)
+    public synchronized List<String> getPartitionReplicatedFiles(int partition, IReplicationStrategy strategy)
             throws HyracksDataException {
         final List<String> partitionReplicatedFiles = new ArrayList<>();
         final Set<File> replicatedIndexes = new HashSet<>();
@@ -438,7 +438,7 @@
         return partitionReplicatedFiles;
     }
 
-    public long getReplicatedIndexesMaxComponentId(int partition, IReplicationStrategy strategy)
+    public synchronized long getReplicatedIndexesMaxComponentId(int partition, IReplicationStrategy strategy)
             throws HyracksDataException {
         long maxComponentId = LSMComponentId.MIN_VALID_COMPONENT_ID;
         final Map<Long, LocalResource> partitionResources = getPartitionResources(partition);
@@ -474,7 +474,7 @@
         }
     }
 
-    public void cleanup(int partition) throws HyracksDataException {
+    public synchronized void cleanup(int partition) throws HyracksDataException {
         final Set<File> partitionIndexes = getPartitionIndexes(partition);
         try {
             for (File index : partitionIndexes) {
@@ -501,7 +501,7 @@
         return resourcesStats;
     }
 
-    public void deleteCorruptedResources() throws HyracksDataException {
+    public synchronized void deleteCorruptedResources() throws HyracksDataException {
         for (Path root : storageRoots) {
             final Collection<File> metadataMaskFiles =
                     FileUtils.listFiles(root.toFile(), METADATA_MASK_FILES_FILTER, ALL_DIR_FILTER);
@@ -648,7 +648,7 @@
         return storageRoots;
     }
 
-    public void keepPartitions(Set<Integer> keepPartitions) {
+    public synchronized void keepPartitions(Set<Integer> keepPartitions) {
         List<File> onDiskPartitions = getOnDiskPartitions();
         for (File onDiskPartition : onDiskPartitions) {
             int partitionNum = StoragePathUtil.getPartitionNumFromRelativePath(onDiskPartition.getAbsolutePath());
@@ -660,7 +660,7 @@
         }
     }
 
-    public List<File> getOnDiskPartitions() {
+    public synchronized List<File> getOnDiskPartitions() {
         List<File> onDiskPartitions = new ArrayList<>();
         for (Path root : storageRoots) {
             File[] partitions = root.toFile().listFiles(

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I95f8565780675798393d7d43c0051c04e2b0a98c
Gerrit-Change-Number: 13164
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-MessageType: newchange

Change in asterixdb[master]: [NO ISSUE][STO] Ensure resources file operations are synchronized

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Murtadha Hubail <mh...@apache.org>:

Murtadha Hubail has removed a vote from this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164 )


Change subject: [NO ISSUE][STO] Ensure resources file operations are synchronized
......................................................................


Removed Contrib-2 by Anon. E. Moose (1000171)
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I95f8565780675798393d7d43c0051c04e2b0a98c
Gerrit-Change-Number: 13164
Gerrit-PatchSet: 4
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-MessageType: deleteVote

Change in asterixdb[master]: [NO ISSUE][STO] Ensure resources file operations are synchronized

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Murtadha Hubail <mh...@apache.org>:

Murtadha Hubail has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164 )


Change subject: [NO ISSUE][STO] Ensure resources file operations are synchronized
......................................................................

[NO ISSUE][STO] Ensure resources file operations are synchronized

- user model changes: no
- storage format changes: no
- interface changes: no

Details:

- To avoid an operation reading a partially written resource file,
  ensure all such operations are synchronized.

Change-Id: I95f8565780675798393d7d43c0051c04e2b0a98c
---
M asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
1 file changed, 14 insertions(+), 14 deletions(-)



  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/64/13164/1

diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 4d15385..06e4015 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -304,7 +304,7 @@
     }
 
     @Override
-    public long maxId() throws HyracksDataException {
+    public synchronized long maxId() throws HyracksDataException {
         final Map<Long, LocalResource> allResources = loadAndGetAllResources();
         final Optional<Long> max = allResources.keySet().stream().max(Long::compare);
         return max.isPresent() ? max.get() : 0;
@@ -330,7 +330,7 @@
         }
     }
 
-    public void setReplicationManager(IReplicationManager replicationManager) {
+    public synchronized void setReplicationManager(IReplicationManager replicationManager) {
         this.replicationManager = replicationManager;
         isReplicationEnabled = replicationManager.isReplicationEnabled();
 
@@ -357,7 +357,7 @@
      *
      * @throws IOException
      */
-    public void deleteStorageData() throws IOException {
+    public synchronized void deleteStorageData() throws IOException {
         for (Path root : storageRoots) {
             final File rootFile = root.toFile();
             if (rootFile.exists()) {
@@ -367,13 +367,13 @@
         createStorageRoots();
     }
 
-    public Set<Integer> getAllPartitions() throws HyracksDataException {
+    public synchronized Set<Integer> getAllPartitions() throws HyracksDataException {
         return loadAndGetAllResources().values().stream().map(LocalResource::getResource)
                 .map(DatasetLocalResource.class::cast).map(DatasetLocalResource::getPartition)
                 .collect(Collectors.toSet());
     }
 
-    public Optional<DatasetResourceReference> getLocalResourceReference(String absoluteFilePath)
+    public synchronized Optional<DatasetResourceReference> getLocalResourceReference(String absoluteFilePath)
             throws HyracksDataException {
         final String localResourcePath = StoragePathUtil.getIndexFileRelativePath(absoluteFilePath);
         final LocalResource lr = get(localResourcePath);
@@ -388,7 +388,7 @@
      * @return The set of indexes files
      * @throws HyracksDataException
      */
-    public Set<File> getPartitionIndexes(int partition) throws HyracksDataException {
+    public synchronized Set<File> getPartitionIndexes(int partition) throws HyracksDataException {
         final Map<Long, LocalResource> partitionResourcesMap = getResources(resource -> {
             DatasetLocalResource dsResource = (DatasetLocalResource) resource.getResource();
             return dsResource.getPartition() == partition;
@@ -400,14 +400,14 @@
         return indexes;
     }
 
-    public Map<Long, LocalResource> getPartitionResources(int partition) throws HyracksDataException {
+    public synchronized Map<Long, LocalResource> getPartitionResources(int partition) throws HyracksDataException {
         return getResources(resource -> {
             DatasetLocalResource dsResource = (DatasetLocalResource) resource.getResource();
             return dsResource.getPartition() == partition;
         });
     }
 
-    public Map<String, Long> getPartitionReplicatedResources(int partition, IReplicationStrategy strategy)
+    public synchronized Map<String, Long> getPartitionReplicatedResources(int partition, IReplicationStrategy strategy)
             throws HyracksDataException {
         final Map<String, Long> partitionReplicatedResources = new HashMap<>();
         final Map<Long, LocalResource> partitionResources = getPartitionResources(partition);
@@ -421,7 +421,7 @@
         return partitionReplicatedResources;
     }
 
-    public List<String> getPartitionReplicatedFiles(int partition, IReplicationStrategy strategy)
+    public synchronized List<String> getPartitionReplicatedFiles(int partition, IReplicationStrategy strategy)
             throws HyracksDataException {
         final List<String> partitionReplicatedFiles = new ArrayList<>();
         final Set<File> replicatedIndexes = new HashSet<>();
@@ -438,7 +438,7 @@
         return partitionReplicatedFiles;
     }
 
-    public long getReplicatedIndexesMaxComponentId(int partition, IReplicationStrategy strategy)
+    public synchronized long getReplicatedIndexesMaxComponentId(int partition, IReplicationStrategy strategy)
             throws HyracksDataException {
         long maxComponentId = LSMComponentId.MIN_VALID_COMPONENT_ID;
         final Map<Long, LocalResource> partitionResources = getPartitionResources(partition);
@@ -474,7 +474,7 @@
         }
     }
 
-    public void cleanup(int partition) throws HyracksDataException {
+    public synchronized void cleanup(int partition) throws HyracksDataException {
         final Set<File> partitionIndexes = getPartitionIndexes(partition);
         try {
             for (File index : partitionIndexes) {
@@ -501,7 +501,7 @@
         return resourcesStats;
     }
 
-    public void deleteCorruptedResources() throws HyracksDataException {
+    public synchronized void deleteCorruptedResources() throws HyracksDataException {
         for (Path root : storageRoots) {
             final Collection<File> metadataMaskFiles =
                     FileUtils.listFiles(root.toFile(), METADATA_MASK_FILES_FILTER, ALL_DIR_FILTER);
@@ -648,7 +648,7 @@
         return storageRoots;
     }
 
-    public void keepPartitions(Set<Integer> keepPartitions) {
+    public synchronized void keepPartitions(Set<Integer> keepPartitions) {
         List<File> onDiskPartitions = getOnDiskPartitions();
         for (File onDiskPartition : onDiskPartitions) {
             int partitionNum = StoragePathUtil.getPartitionNumFromRelativePath(onDiskPartition.getAbsolutePath());
@@ -660,7 +660,7 @@
         }
     }
 
-    public List<File> getOnDiskPartitions() {
+    public synchronized List<File> getOnDiskPartitions() {
         List<File> onDiskPartitions = new ArrayList<>();
         for (Path root : storageRoots) {
             File[] partitions = root.toFile().listFiles(

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I95f8565780675798393d7d43c0051c04e2b0a98c
Gerrit-Change-Number: 13164
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-MessageType: newchange

Change in asterixdb[master]: [NO ISSUE][STO] Ensure resources file operations are synchronized

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Jenkins <je...@fulliautomatix.ics.uci.edu>:

Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164 )

Change subject: [NO ISSUE][STO] Ensure resources file operations are synchronized
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/12465/ : SUCCESS


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I95f8565780675798393d7d43c0051c04e2b0a98c
Gerrit-Change-Number: 13164
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-Comment-Date: Sat, 11 Sep 2021 00:29:51 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[master]: [NO ISSUE][STO] Ensure resources file operations are synchronized

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
Anon. E. Moose #1000171 has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164 )

Change subject: [NO ISSUE][STO] Ensure resources file operations are synchronized
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/SuYD7TEuUzWZiA9Y8 : SUCCESS


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I95f8565780675798393d7d43c0051c04e2b0a98c
Gerrit-Change-Number: 13164
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-CC: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Fri, 10 Sep 2021 22:47:18 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: No
Gerrit-MessageType: comment

Change in asterixdb[master]: [NO ISSUE][STO] Ensure resources file operations are synchronized

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Murtadha Hubail <mh...@apache.org>:

Murtadha Hubail has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164 )

Change subject: [NO ISSUE][STO] Ensure resources file operations are synchronized
......................................................................


Patch Set 4: Code-Review+1


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I95f8565780675798393d7d43c0051c04e2b0a98c
Gerrit-Change-Number: 13164
Gerrit-PatchSet: 4
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Comment-Date: Sun, 12 Sep 2021 09:48:56 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[master]: [NO ISSUE][STO] Ensure resources file operations are synchronized

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
Anon. E. Moose #1000171 has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164 )

Change subject: [NO ISSUE][STO] Ensure resources file operations are synchronized
......................................................................


Patch Set 2:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/oiwDKDVBDbiGJaTUA : SUCCESS


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I95f8565780675798393d7d43c0051c04e2b0a98c
Gerrit-Change-Number: 13164
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Sat, 11 Sep 2021 19:35:18 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: No
Gerrit-MessageType: comment

Change in asterixdb[master]: [NO ISSUE][STO] Ensure resources file operations are synchronized

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Murtadha Hubail <mh...@apache.org>:

Hello Jenkins, Anon. E. Moose #1000171, 

I'd like you to reexamine a change. Please visit

    https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164

to look at the new patch set (#4).

Change subject: [NO ISSUE][STO] Ensure resources file operations are synchronized
......................................................................

[NO ISSUE][STO] Ensure resources file operations are synchronized

- user model changes: no
- storage format changes: no
- interface changes: no

Details:

- To avoid an operation reading a partially written resource file,
  ensure all such operations are synchronized.
- Limit partition resources search to the partition's root directory.

Change-Id: I95f8565780675798393d7d43c0051c04e2b0a98c
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
M asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsNotifier.java
M asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
M asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
4 files changed, 60 insertions(+), 34 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/64/13164/4
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I95f8565780675798393d7d43c0051c04e2b0a98c
Gerrit-Change-Number: 13164
Gerrit-PatchSet: 4
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-MessageType: newpatchset

Change in asterixdb[master]: [NO ISSUE][STO] Ensure resources file operations are synchronized

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
Anon. E. Moose #1000171 has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164 )

Change subject: [NO ISSUE][STO] Ensure resources file operations are synchronized
......................................................................


Patch Set 1: Contrib-2

Analytics Compatibility Tests Failed
https://cbjenkins.page.link/RddobzVoE33npSoM8 : UNSTABLE


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I95f8565780675798393d7d43c0051c04e2b0a98c
Gerrit-Change-Number: 13164
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Sat, 11 Sep 2021 02:19:39 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[master]: [NO ISSUE][STO] Ensure resources file operations are synchronized

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Ali Alsuliman <al...@gmail.com>:

Ali Alsuliman has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164 )

Change subject: [NO ISSUE][STO] Ensure resources file operations are synchronized
......................................................................


Patch Set 4: Code-Review+2


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I95f8565780675798393d7d43c0051c04e2b0a98c
Gerrit-Change-Number: 13164
Gerrit-PatchSet: 4
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Comment-Date: Mon, 13 Sep 2021 16:22:45 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[master]: [NO ISSUE][STO] Ensure resources file operations are synchronized

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Murtadha Hubail <mh...@apache.org>:

Hello Jenkins, Anon. E. Moose #1000171, 

I'd like you to reexamine a change. Please visit

    https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164

to look at the new patch set (#2).

Change subject: [NO ISSUE][STO] Ensure resources file operations are synchronized
......................................................................

[NO ISSUE][STO] Ensure resources file operations are synchronized

- user model changes: no
- storage format changes: no
- interface changes: no

Details:

- To avoid an operation reading a partially written resource file,
  ensure all such operations are synchronized.
- Limit partition resources search to the partition's root directory.
- Ensure no resource files are being read while other are being
  read/deleted.

Change-Id: I95f8565780675798393d7d43c0051c04e2b0a98c
---
M asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DropIndexTask.java
M asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
M asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
3 files changed, 63 insertions(+), 33 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/64/13164/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13164
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I95f8565780675798393d7d43c0051c04e2b0a98c
Gerrit-Change-Number: 13164
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-MessageType: newpatchset