[
https://issues.apache.org/jira/browse/HADOOP-19236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17902486#comment-17902486
]
ASF GitHub Bot commented on HADOOP-19236:
-----------------------------------------
wojiaodoubao commented on code in PR #7194:
URL: https://github.com/apache/hadoop/pull/7194#discussion_r1866879802
##########
hadoop-cloud-storage-project/hadoop-tos/hadoop-tos-core/src/main/java/org/apache/hadoop/fs/tosfs/object/FileStore.java:
##########
@@ -0,0 +1,633 @@
+/*
+ * 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.hadoop.fs.tosfs.object;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.tosfs.conf.ConfKeys;
+import org.apache.hadoop.fs.tosfs.conf.FileStoreKeys;
+import org.apache.hadoop.fs.tosfs.object.exceptions.NotAppendableException;
+import org.apache.hadoop.fs.tosfs.object.request.ListObjectsRequest;
+import org.apache.hadoop.fs.tosfs.object.response.ListObjectsResponse;
+import org.apache.hadoop.fs.tosfs.util.CommonUtils;
+import org.apache.hadoop.fs.tosfs.util.Range;
+import org.apache.hadoop.fs.tosfs.util.UUIDUtils;
+import org.apache.hadoop.thirdparty.com.google.common.base.Strings;
+import org.apache.hadoop.util.Lists;
+import org.apache.hadoop.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public class FileStore implements ObjectStorage {
Review Comment:
I'll recommend to keep the FileStore, so all the unit tests could run
independently from TOS. Currently the hadoop-tos module still depends on TOS to
run most test cases. But in the future, my plan is to switch them to both
FileStore and TOS, then we can test without TOS.
> Integration of Volcano Engine TOS in Hadoop.
> --------------------------------------------
>
> Key: HADOOP-19236
> URL: https://issues.apache.org/jira/browse/HADOOP-19236
> Project: Hadoop Common
> Issue Type: New Feature
> Components: fs, tools
> Affects Versions: 3.4.0
> Reporter: Jinglun
> Assignee: Zheng Hu
> Priority: Major
> Labels: pull-request-available
> Attachments: Integration of Volcano Engine TOS in Hadoop.pdf
>
>
> Volcano Engine is a fast growing cloud vendor launched by ByteDance, and TOS
> is the object storage service of Volcano Engine. A common way is to store
> data into TOS and run Hadoop/Spark/Flink applications to access TOS. But
> there is no original support for TOS in hadoop, thus it is not easy for users
> to build their Big Data System based on TOS.
>
> This work aims to integrate TOS with Hadoop to help users run their
> applications on TOS. Users only need to do some simple configuration, then
> their applications can read/write TOS without any code change. This work is
> similar to AWS S3, AzureBlob, AliyunOSS, Tencnet COS and HuaweiCloud Object
> Storage in Hadoop.
>
> Please see the attached document "Integration of Volcano Engine TOS in
> Hadoop" for more details.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]