zuston commented on code in PR #2253:
URL:
https://github.com/apache/incubator-uniffle/pull/2253#discussion_r1853499347
##########
common/src/main/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitter.java:
##########
@@ -112,27 +130,34 @@ public List<ShuffleDataSegment> split(ShuffleIndexResult
shuffleIndexResult) {
blockId);
break;
}
-
boolean conditionOfDiscontinuousBlocks =
lastExpectedBlockIndex != -1
&& bufferSegments.size() > 0
&& expectTaskIds.contains(taskAttemptId)
&& index - lastExpectedBlockIndex != 1;
boolean conditionOfLimitedBufferSize = bufferOffset >= readBufferSize;
-
- if (conditionOfDiscontinuousBlocks || conditionOfLimitedBufferSize) {
- ShuffleDataSegment sds = new ShuffleDataSegment(fileOffset,
bufferOffset, bufferSegments);
+ boolean storageChanged =
Review Comment:
ditto
##########
common/src/main/java/org/apache/uniffle/common/segment/FixedSizeSegmentSplitter.java:
##########
@@ -91,24 +102,35 @@ private static List<ShuffleDataSegment>
transIndexDataToSegments(
break;
}
- bufferSegments.add(
- new BufferSegment(blockId, bufferOffset, length, uncompressLength,
crc, taskAttemptId));
- bufferOffset += length;
+ boolean storageChanged = preStorageId != -1 && currentStorageId !=
preStorageId;
Review Comment:
This class should be fully tested by unit test cases to cover the storage
changed situation
##########
server/src/main/java/org/apache/uniffle/server/ShuffleServerConf.java:
##########
@@ -744,6 +745,12 @@ public class ShuffleServerConf extends RssBaseConf {
.defaultValue(false)
.withDescription("Whether to enable app detail log");
+ public static final ConfigOption<String> SERVER_LOCAL_STORAGE_MANAGER_CLASS =
+ ConfigOptions.key("rss.server.localStorageManagerClass")
+ .stringType()
+ .defaultValue(MultiPartLocalStorageManager.class.getName())
Review Comment:
I hope we could reserve the preivous manager as the default value.
##########
common/src/main/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitter.java:
##########
@@ -85,7 +85,12 @@ public List<ShuffleDataSegment> split(ShuffleIndexResult
shuffleIndexResult) {
* ShuffleDataSegment size should < readBufferSize 3. Single
shuffleDataSegment's blocks should
* be continuous
*/
+ int[] storageIds = shuffleIndexResult.getStorageIds();
Review Comment:
For the fixedSizeSegments and localOrderSegmentSpilter, the storage changing
logic may be unified into the underlying abstract segment spiltter.
##########
server/src/main/java/org/apache/uniffle/server/storage/MultiPartLocalStorageManager.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.uniffle.server.storage;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.uniffle.server.ShuffleDataFlushEvent;
+import org.apache.uniffle.server.ShuffleDataReadEvent;
+import org.apache.uniffle.server.ShuffleServerConf;
+import org.apache.uniffle.storage.common.CompositeStorage;
+import org.apache.uniffle.storage.common.LocalStorage;
+import org.apache.uniffle.storage.common.Storage;
+import org.apache.uniffle.storage.util.ShuffleStorageUtils;
+
+public class MultiPartLocalStorageManager extends LocalStorageManager {
+ private static final Logger LOG =
LoggerFactory.getLogger(MultiPartLocalStorageManager.class);
+ // id -> storage
+ private final Map<Integer, LocalStorage> idToStorages;
+
+ private final CompositeStorage compositeStorage;
+
+ public MultiPartLocalStorageManager(ShuffleServerConf conf) {
+ super(conf);
+ idToStorages = new ConcurrentSkipListMap<>();
+ for (LocalStorage storage : getStorages()) {
+ idToStorages.put(storage.getId(), storage);
+ }
+
+ compositeStorage = new CompositeStorage(getStorages());
+ }
+
+ @Override
+ public Storage selectStorage(ShuffleDataFlushEvent event) {
+ if (getStorages().size() == 1) {
+ if (event.getUnderStorage() == null) {
+ event.setUnderStorage(getStorages().get(0));
+ }
+ return getStorages().get(0);
+ }
+ String appId = event.getAppId();
+ int shuffleId = event.getShuffleId();
+ int partitionId = event.getStartPartition();
+
+ // TODO(baoloongmao): extend to support select storage by free space
+ // eventId is a non-negative long.
+ LocalStorage storage = getStorages().get((int) (event.getEventId() %
getStorages().size()));
+ if (storage != null) {
+ if (storage.isCorrupted()) {
+ if (storage.containsWriteHandler(appId, shuffleId, partitionId)) {
+ LOG.error(
+ "LocalStorage: {} is corrupted. Switching another storage for
event: {}, some data will be lost",
+ storage.getBasePath(),
+ event);
+ }
+ } else {
+ if (event.getUnderStorage() == null) {
+ event.setUnderStorage(storage);
+ }
+ return storage;
+ }
+ }
+
+ // TODO(baoloongmao): update health storages and store it as member of
this class.
+ List<LocalStorage> candidates =
+ getStorages().stream()
+ .filter(x -> x.canWrite() && !x.isCorrupted())
+ .collect(Collectors.toList());
+
+ if (candidates.size() == 0) {
+ return null;
+ }
+ final LocalStorage selectedStorage =
+ candidates.get(
+ ShuffleStorageUtils.getStorageIndex(candidates.size(), appId,
shuffleId, partitionId));
+ if (storage == null || storage.isCorrupted() || event.getUnderStorage() ==
null) {
+ event.setUnderStorage(selectedStorage);
+ return selectedStorage;
+ }
+ return storage;
+ }
+
+ @Override
+ public Storage selectStorage(ShuffleDataReadEvent event) {
+ if (getStorages().size() == 1) {
+ return getStorages().get(0);
+ }
+
+ // Use higher 8 bit to storage the storage id, and use lower 56 bit to
storage the offset.
+ int storageId = event.getStorageId();
+ // TODO(baoloongmao): check AOOB exception
+ return idToStorages.get(storageId);
+ }
+
+ @Override
+ public Storage selectStorageForIndex(ShuffleDataReadEvent event) {
Review Comment:
How about renaming to `selectStorageById` ?
##########
server/src/main/java/org/apache/uniffle/server/storage/MultiPartLocalStorageManager.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.uniffle.server.storage;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.uniffle.server.ShuffleDataFlushEvent;
+import org.apache.uniffle.server.ShuffleDataReadEvent;
+import org.apache.uniffle.server.ShuffleServerConf;
+import org.apache.uniffle.storage.common.CompositeStorage;
+import org.apache.uniffle.storage.common.LocalStorage;
+import org.apache.uniffle.storage.common.Storage;
+import org.apache.uniffle.storage.util.ShuffleStorageUtils;
+
+public class MultiPartLocalStorageManager extends LocalStorageManager {
+ private static final Logger LOG =
LoggerFactory.getLogger(MultiPartLocalStorageManager.class);
+ // id -> storage
+ private final Map<Integer, LocalStorage> idToStorages;
+
+ private final CompositeStorage compositeStorage;
+
+ public MultiPartLocalStorageManager(ShuffleServerConf conf) {
+ super(conf);
+ idToStorages = new ConcurrentSkipListMap<>();
+ for (LocalStorage storage : getStorages()) {
+ idToStorages.put(storage.getId(), storage);
+ }
+
+ compositeStorage = new CompositeStorage(getStorages());
+ }
+
+ @Override
+ public Storage selectStorage(ShuffleDataFlushEvent event) {
+ if (getStorages().size() == 1) {
+ if (event.getUnderStorage() == null) {
+ event.setUnderStorage(getStorages().get(0));
+ }
+ return getStorages().get(0);
+ }
+ String appId = event.getAppId();
+ int shuffleId = event.getShuffleId();
+ int partitionId = event.getStartPartition();
+
+ // TODO(baoloongmao): extend to support select storage by free space
+ // eventId is a non-negative long.
+ LocalStorage storage = getStorages().get((int) (event.getEventId() %
getStorages().size()));
+ if (storage != null) {
+ if (storage.isCorrupted()) {
+ if (storage.containsWriteHandler(appId, shuffleId, partitionId)) {
+ LOG.error(
+ "LocalStorage: {} is corrupted. Switching another storage for
event: {}, some data will be lost",
+ storage.getBasePath(),
+ event);
+ }
+ } else {
+ if (event.getUnderStorage() == null) {
+ event.setUnderStorage(storage);
+ }
+ return storage;
+ }
+ }
+
+ // TODO(baoloongmao): update health storages and store it as member of
this class.
+ List<LocalStorage> candidates =
+ getStorages().stream()
+ .filter(x -> x.canWrite() && !x.isCorrupted())
+ .collect(Collectors.toList());
+
+ if (candidates.size() == 0) {
+ return null;
+ }
+ final LocalStorage selectedStorage =
+ candidates.get(
+ ShuffleStorageUtils.getStorageIndex(candidates.size(), appId,
shuffleId, partitionId));
+ if (storage == null || storage.isCorrupted() || event.getUnderStorage() ==
null) {
+ event.setUnderStorage(selectedStorage);
+ return selectedStorage;
+ }
+ return storage;
+ }
+
+ @Override
+ public Storage selectStorage(ShuffleDataReadEvent event) {
+ if (getStorages().size() == 1) {
+ return getStorages().get(0);
+ }
+
+ // Use higher 8 bit to storage the storage id, and use lower 56 bit to
storage the offset.
Review Comment:
Remove this comment, which is not consistent with the final design
##########
storage/src/main/java/org/apache/uniffle/storage/common/CompositeStorage.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.uniffle.storage.common;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.uniffle.common.exception.FileNotFoundException;
+import org.apache.uniffle.storage.handler.api.ServerReadHandler;
+import org.apache.uniffle.storage.handler.api.ShuffleWriteHandler;
+import
org.apache.uniffle.storage.handler.impl.CompositeLocalFileServerReadHandler;
+import org.apache.uniffle.storage.request.CreateShuffleReadHandlerRequest;
+import org.apache.uniffle.storage.request.CreateShuffleWriteHandlerRequest;
+
+public class CompositeStorage extends AbstractStorage {
Review Comment:
From my sight, this wrapper is just as a consistency view for the invoking
side when reading. So how about renaming this by `CompositeReadingViewStorage`
?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]