drill-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] [drill] cgivre commented on a change in pull request #2084: DRILL-7745: Add storage plugin for IPFS
Date Tue, 23 Jun 2020 01:45:30 GMT

cgivre commented on a change in pull request #2084:
URL: https://github.com/apache/drill/pull/2084#discussion_r443908319



##########
File path: contrib/storage-ipfs/README.md
##########
@@ -0,0 +1,182 @@
+# Drill Storage Plugin for IPFS
+
+[中文](README.zh.md)
+
+## Contents
+
+0. [Introduction](#Introduction)
+1. [Compile](#Compile)
+2. [Install](#Install)
+2. [Configuration](#Configuration)
+3. [Run](#Run)
+
+## Introduction
+
+Minerva is a storage plugin of Drill that connects IPFS's decentralized storage and Drill's flexible query engine. Any data file stored on IPFS can be easily accessed from Drill's query interface, just like a file stored on a local disk. Moreover, with Drill's capability of distributed execution, other instances who are also running Minerva can help accelerate the execution: the data stays where it was, and the queries go to the most suitable nodes which stores the data locally and from there the operations can be performed most efficiently. 
+
+Slides that explain our ideas and the technical details of Minerva: <https://www.slideshare.net/BowenDing4/minerva-ipfs-storage-plugin-for-ipfs>
+
+A live demo: <http://www.datahub.pub/> hosted on a private cluster of Minerva.
+
+Note that it's still in early stages of development and the overall stability and performance is not satisfactory. PRs are very much welcome!

Review comment:
       When we're ready to commit, can you please update the docs and remove all the language about building Drill etc. 

##########
File path: contrib/storage-ipfs/src/main/java/org/apache/drill/exec/store/ipfs/IPFSGroupScan.java
##########
@@ -0,0 +1,456 @@
+/*
+ * Copyright (c) 2018-2020 Bowen Ding, Yuedong Xu, Liang Wang
+ *
+ * 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.drill.exec.store.ipfs;
+
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.shaded.guava.com.google.common.cache.LoadingCache;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.collect.ArrayListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.ListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import io.ipfs.api.MerkleNode;
+import io.ipfs.multihash.Multihash;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.util.DrillVersionInfo;
+import org.apache.drill.exec.coord.ClusterCoordinator;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.schedule.AffinityCreator;
+import org.apache.drill.exec.store.schedule.AssignmentCreator;
+import org.apache.drill.exec.store.schedule.CompleteWork;
+import org.apache.drill.exec.store.schedule.EndpointByteMap;
+import org.apache.drill.exec.store.schedule.EndpointByteMapImpl;
+import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.RecursiveTask;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.drill.exec.store.ipfs.IPFSStoragePluginConfig.IPFSTimeOut.FETCH_DATA;
+
+@JsonTypeName("ipfs-scan")
+public class IPFSGroupScan extends AbstractGroupScan {
+  private static final Logger logger = LoggerFactory.getLogger(IPFSGroupScan.class);
+  private IPFSContext ipfsContext;
+  private IPFSScanSpec ipfsScanSpec;
+  private IPFSStoragePluginConfig config;
+  private List<SchemaPath> columns;
+
+  private static long DEFAULT_NODE_SIZE = 1000l;
+
+  private ListMultimap<Integer, IPFSWork> assignments;
+  private List<IPFSWork> ipfsWorkList = Lists.newArrayList();
+  private Map<String, List<IPFSWork>> endpointWorksMap;
+  private List<EndpointAffinity> affinities;
+
+  @JsonCreator
+  public IPFSGroupScan(@JsonProperty("IPFSScanSpec") IPFSScanSpec ipfsScanSpec,
+                       @JsonProperty("IPFSStoragePluginConfig") IPFSStoragePluginConfig ipfsStoragePluginConfig,
+                       @JsonProperty("columns") List<SchemaPath> columns,
+                       @JacksonInject StoragePluginRegistry pluginRegistry) throws IOException, ExecutionSetupException {
+    this(
+        ((IPFSStoragePlugin) pluginRegistry.getPlugin(ipfsStoragePluginConfig)).getIPFSContext(),
+        ipfsScanSpec,
+        columns
+    );
+  }
+
+  public IPFSGroupScan(IPFSContext ipfsContext,
+                       IPFSScanSpec ipfsScanSpec,
+                       List<SchemaPath> columns) {
+    super((String) null);
+    this.ipfsContext = ipfsContext;
+    this.ipfsScanSpec = ipfsScanSpec;
+    this.config = ipfsContext.getStoragePluginConfig();
+    logger.debug("GroupScan constructor called with columns {}", columns);
+    this.columns = columns == null || columns.size() == 0? ALL_COLUMNS : columns;
+    init();
+  }
+
+  private void init() {
+    IPFSHelper ipfsHelper = ipfsContext.getIPFSHelper();
+    ipfsHelper.setMaxPeersPerLeaf(config.getMaxNodesPerLeaf());
+    ipfsHelper.setTimeouts(config.getIpfsTimeouts());
+    endpointWorksMap = new HashMap<>();
+
+    Multihash topHash = ipfsScanSpec.getTargetHash(ipfsHelper);
+    LoadingCache<Multihash, IPFSPeer> peerMap = ipfsContext.getIPFSPeerCache();
+
+    try {
+      //TODO detect and warn about loops/recursions in a malformed tree

Review comment:
       Please either remove `TODO` OR leave it and reference a JIRA. 

##########
File path: contrib/storage-ipfs/src/main/java/org/apache/drill/exec/store/ipfs/IPFSGroupScan.java
##########
@@ -0,0 +1,456 @@
+/*
+ * Copyright (c) 2018-2020 Bowen Ding, Yuedong Xu, Liang Wang
+ *
+ * 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.drill.exec.store.ipfs;
+
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.shaded.guava.com.google.common.cache.LoadingCache;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.collect.ArrayListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.ListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import io.ipfs.api.MerkleNode;
+import io.ipfs.multihash.Multihash;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.util.DrillVersionInfo;
+import org.apache.drill.exec.coord.ClusterCoordinator;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.schedule.AffinityCreator;
+import org.apache.drill.exec.store.schedule.AssignmentCreator;
+import org.apache.drill.exec.store.schedule.CompleteWork;
+import org.apache.drill.exec.store.schedule.EndpointByteMap;
+import org.apache.drill.exec.store.schedule.EndpointByteMapImpl;
+import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.RecursiveTask;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.drill.exec.store.ipfs.IPFSStoragePluginConfig.IPFSTimeOut.FETCH_DATA;
+
+@JsonTypeName("ipfs-scan")
+public class IPFSGroupScan extends AbstractGroupScan {
+  private static final Logger logger = LoggerFactory.getLogger(IPFSGroupScan.class);
+  private IPFSContext ipfsContext;
+  private IPFSScanSpec ipfsScanSpec;
+  private IPFSStoragePluginConfig config;
+  private List<SchemaPath> columns;
+
+  private static long DEFAULT_NODE_SIZE = 1000l;
+
+  private ListMultimap<Integer, IPFSWork> assignments;
+  private List<IPFSWork> ipfsWorkList = Lists.newArrayList();
+  private Map<String, List<IPFSWork>> endpointWorksMap;
+  private List<EndpointAffinity> affinities;
+
+  @JsonCreator
+  public IPFSGroupScan(@JsonProperty("IPFSScanSpec") IPFSScanSpec ipfsScanSpec,
+                       @JsonProperty("IPFSStoragePluginConfig") IPFSStoragePluginConfig ipfsStoragePluginConfig,
+                       @JsonProperty("columns") List<SchemaPath> columns,
+                       @JacksonInject StoragePluginRegistry pluginRegistry) throws IOException, ExecutionSetupException {
+    this(
+        ((IPFSStoragePlugin) pluginRegistry.getPlugin(ipfsStoragePluginConfig)).getIPFSContext(),
+        ipfsScanSpec,
+        columns
+    );
+  }
+
+  public IPFSGroupScan(IPFSContext ipfsContext,
+                       IPFSScanSpec ipfsScanSpec,
+                       List<SchemaPath> columns) {
+    super((String) null);
+    this.ipfsContext = ipfsContext;
+    this.ipfsScanSpec = ipfsScanSpec;
+    this.config = ipfsContext.getStoragePluginConfig();
+    logger.debug("GroupScan constructor called with columns {}", columns);
+    this.columns = columns == null || columns.size() == 0? ALL_COLUMNS : columns;
+    init();
+  }
+
+  private void init() {
+    IPFSHelper ipfsHelper = ipfsContext.getIPFSHelper();
+    ipfsHelper.setMaxPeersPerLeaf(config.getMaxNodesPerLeaf());
+    ipfsHelper.setTimeouts(config.getIpfsTimeouts());
+    endpointWorksMap = new HashMap<>();
+
+    Multihash topHash = ipfsScanSpec.getTargetHash(ipfsHelper);
+    LoadingCache<Multihash, IPFSPeer> peerMap = ipfsContext.getIPFSPeerCache();
+
+    try {
+      //TODO detect and warn about loops/recursions in a malformed tree
+      class IPFSTreeFlattener extends RecursiveTask<Map<Multihash, String>> {
+        private Multihash hash;
+        private boolean isProvider;
+        private Map<Multihash, String> ret = new LinkedHashMap<>();
+
+        public IPFSTreeFlattener(Multihash hash, boolean isProvider) {
+          this.hash = hash;
+          this.isProvider = isProvider;
+        }
+
+        @Override
+        public Map<Multihash, String> compute() {
+          try {
+            if (isProvider) {
+              IPFSPeer peer = peerMap.getUnchecked(hash);
+              ret.put(hash, peer.hasDrillbitAddress() ? peer.getDrillbitAddress().get() : null);
+              return ret;
+            }
+
+            MerkleNode metaOrSimpleNode = ipfsHelper.timedFailure(ipfsHelper.getClient().object::links, hash, config.getIpfsTimeout(FETCH_DATA));
+            if (metaOrSimpleNode.links.size() > 0) {
+              logger.debug("{} is a meta node", hash);
+              //TODO do something useful with leaf size, e.g. hint Drill about operation costs
+              List<Multihash> intermediates = metaOrSimpleNode.links.stream().map(x -> x.hash).collect(Collectors.toList());
+
+              ImmutableList.Builder<IPFSTreeFlattener> builder = ImmutableList.builder();
+              for (Multihash intermediate : intermediates.subList(1, intermediates.size())) {
+                builder.add(new IPFSTreeFlattener(intermediate, false));
+              }
+              ImmutableList<IPFSTreeFlattener> subtasks = builder.build();
+              subtasks.forEach(IPFSTreeFlattener::fork);
+
+              IPFSTreeFlattener first = new IPFSTreeFlattener(intermediates.get(0), false);
+              ret.putAll(first.compute());
+              subtasks.reverse().forEach(
+                  subtask -> ret.putAll(subtask.join())
+              );
+
+            } else {
+              logger.debug("{} is a simple node", hash);
+              List<IPFSPeer> providers = ipfsHelper.findprovsTimeout(hash).stream()
+                  .map(id ->
+                    peerMap.getUnchecked(id)
+                  )
+                  .collect(Collectors.toList());
+              //FIXME isDrillReady may block threads
+              providers = providers.stream()
+                  .filter(IPFSPeer::isDrillReady)
+                  .collect(Collectors.toList());
+              if (providers.size() < 1) {
+                logger.warn("No drill-ready provider found for leaf {}, adding foreman as the provider", hash);
+                providers.add(ipfsContext.getMyself());
+              }
+
+              logger.debug("Got {} providers for {} from IPFS", providers.size(), hash);
+              ImmutableList.Builder<IPFSTreeFlattener> builder = ImmutableList.builder();
+              for (IPFSPeer provider : providers.subList(1, providers.size())) {
+                builder.add(new IPFSTreeFlattener(provider.getId(), true));
+              }
+              ImmutableList<IPFSTreeFlattener> subtasks = builder.build();
+              subtasks.forEach(IPFSTreeFlattener::fork);
+
+              List<String> possibleAddrs = new LinkedList<>();
+              Multihash firstProvider = providers.get(0).getId();
+              IPFSTreeFlattener firstTask = new IPFSTreeFlattener(firstProvider, true);
+              String firstAddr = firstTask.compute().get(firstProvider);
+              if (firstAddr != null) {
+                possibleAddrs.add(firstAddr);
+              }
+
+              subtasks.reverse().forEach(
+                  subtask -> {
+                    String addr = subtask.join().get(subtask.hash);
+                    if (addr != null) {
+                      possibleAddrs.add(addr);
+                    }
+                  }
+              );
+
+              if (possibleAddrs.size() < 1) {
+                logger.error("All attempts to find an appropriate provider address for {} have failed", hash);
+                throw new RuntimeException("No address found for any provider for leaf " + hash);
+              } else {
+                Random random = new Random();
+                String chosenAddr = possibleAddrs.get(random.nextInt(possibleAddrs.size()));
+                ret.clear();
+                ret.put(hash, chosenAddr);
+                logger.debug("Got peer host {} for leaf {}", chosenAddr, hash);
+              }
+            }
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+          return ret;
+        }
+      }
+
+      logger.debug("start to recursively expand nested IPFS hashes, topHash={}", topHash);
+
+      Stopwatch watch = Stopwatch.createStarted();
+      //FIXME parallelization width magic number, maybe a config entry?
+      ForkJoinPool forkJoinPool = new ForkJoinPool(config.getNumWorkerThreads());
+      IPFSTreeFlattener topTask = new IPFSTreeFlattener(topHash, false);
+      Map<Multihash, String> leafAddrMap = forkJoinPool.invoke(topTask);
+
+      logger.debug("Took {} ms to expand hash leaves", watch.elapsed(TimeUnit.MILLISECONDS));
+      logger.debug("Iterating on {} leaves...", leafAddrMap.size());
+      ClusterCoordinator coordinator = ipfsContext.getStoragePlugin().getContext().getClusterCoordinator();
+      for (Multihash leaf : leafAddrMap.keySet()) {
+        String peerHostname = leafAddrMap.get(leaf);
+
+        Optional<DrillbitEndpoint> oep = coordinator.getAvailableEndpoints()
+            .stream()
+            .filter(a -> a.getAddress().equals(peerHostname))
+            .findAny();
+        DrillbitEndpoint ep;
+        if (oep.isPresent()) {
+          ep = oep.get();
+          logger.debug("Using existing endpoint {}", ep.getAddress());
+        } else {
+          logger.debug("created new endpoint on the fly {}", peerHostname);
+          //TODO read ports & version info from IPFS instead of hard-coded
+          ep = DrillbitEndpoint.newBuilder()
+              .setAddress(peerHostname)
+              .setUserPort(31010)
+              .setControlPort(31011)
+              .setDataPort(31012)
+              .setHttpPort(8047)
+              .setVersion(DrillVersionInfo.getVersion())
+              .setState(DrillbitEndpoint.State.ONLINE)
+              .build();
+          //TODO how to safely remove endpoints that are no longer needed once the query is completed?
+          ClusterCoordinator.RegistrationHandle handle = coordinator.register(ep);
+        }
+
+        IPFSWork work = new IPFSWork(leaf.toBase58());
+        logger.debug("added endpoint {} to work {}", ep.getAddress(), work);
+        work.getByteMap().add(ep, DEFAULT_NODE_SIZE);
+        work.setOnEndpoint(ep);
+
+        if(endpointWorksMap.containsKey(ep.getAddress())) {
+          endpointWorksMap.get(ep.getAddress()).add(work);
+        } else {
+          List<IPFSWork> ipfsWorks = Lists.newArrayList();
+          ipfsWorks.add(work);
+          endpointWorksMap.put(ep.getAddress(), ipfsWorks);
+        }
+        ipfsWorkList.add(work);
+      }
+    }catch (Exception e) {
+      logger.debug("exception in init");
+      throw new RuntimeException(e);
+    }
+  }
+
+  private IPFSGroupScan(IPFSGroupScan that) {
+    super(that);
+    this.ipfsContext = that.ipfsContext;
+    this.ipfsScanSpec = that.ipfsScanSpec;
+    this.config = that.config;
+    this.assignments = that.assignments;
+    this.ipfsWorkList = that.ipfsWorkList;
+    this.endpointWorksMap = that.endpointWorksMap;
+    this.columns = that.columns;
+  }
+
+  @JsonProperty
+  public List<SchemaPath> getColumns() {
+    return columns;
+  }
+
+  @JsonIgnore
+  public IPFSStoragePlugin getStoragePlugin() {
+    return ipfsContext.getStoragePlugin();
+  }
+
+  @JsonProperty
+  public IPFSScanSpec getIPFSScanSpec() {
+    return ipfsScanSpec;
+  }
+
+  @Override
+  public List<EndpointAffinity> getOperatorAffinity() {
+    if (affinities == null) {
+      affinities = AffinityCreator.getAffinityMap(ipfsWorkList);
+    }
+    return affinities;
+  }
+
+  @Override
+  public int getMaxParallelizationWidth() {
+    DrillbitEndpoint myself = ipfsContext.getStoragePlugin().getContext().getEndpoint();
+    int width;
+    if (endpointWorksMap.containsKey(myself.getAddress())) {
+      // the foreman is also going to be a minor fragment worker under a UnionExchange operator
+      width = ipfsWorkList.size();
+    } else {
+      // the foreman does not hold data, so we have to force parallelization
+      // to make sure there is a UnionExchange operator
+      width = ipfsWorkList.size() + 1;
+    }
+    logger.debug("getMaxParallelizationWidth: {}", width);
+    return width;
+  }
+
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
+    logger.debug("ipfsWorkList.size() = {}", ipfsWorkList.size());
+    logger.debug("endpointWorksMap: {}", endpointWorksMap);
+    if (endpointWorksMap.size()>1) { //偶尔还会出错?
+      //incomingEndpoints是已经排好顺序的endpoints,和fragment 顺序对应
+      logger.debug("Use manual assignment");
+      assignments = ArrayListMultimap.create();
+      for (int fragmentId = 0; fragmentId < incomingEndpoints.size(); fragmentId++) {
+        String address = incomingEndpoints.get(fragmentId).getAddress();
+        if (endpointWorksMap.containsKey(address)) { //如果对应的节点有工作
+          for (IPFSWork work : endpointWorksMap.get(address)) {
+            assignments.put(fragmentId, work);
+          }
+        } else //如果对应的节点没有工作安排,分配一个空work
+        {
+
+        }
+      }
+    }
+    else //如果出问题,按照系统默认分配模式?
+    {
+     logger.debug("Use AssignmentCreator");
+      assignments = AssignmentCreator.getMappings(incomingEndpoints, ipfsWorkList);
+    }
+
+    for (int i = 0; i < incomingEndpoints.size(); i++) {
+      logger.debug("Fragment {} on endpoint {} is assigned with works: {}", i, incomingEndpoints.get(i).getAddress(), assignments.get(i));

Review comment:
       This seems like it is just generating log messages.  Should this be doing something else?

##########
File path: contrib/storage-ipfs/src/main/java/org/apache/drill/exec/store/ipfs/IPFSCompat.java
##########
@@ -0,0 +1,202 @@
+/*
+ * Copyright (c) 2018-2020 Bowen Ding, Yuedong Xu, Liang Wang
+ *
+ * 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.drill.exec.store.ipfs;
+
+import io.ipfs.api.JSONParser;
+import io.ipfs.multihash.Multihash;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+/*
+ * Compatibility fixes for java-ipfs-http-client library
+ */
+public class IPFSCompat {
+  public final String host;
+  public final int port;
+  private final String version;
+  public final String protocol;
+  public final int readTimeout;
+  public static final int DEFAULT_READ_TIMEOUT = 0;
+
+  public final DHT dht = new DHT();
+  public final Name name = new Name();
+
+  public IPFSCompat(String host, int port) {
+    this(host, port, "/api/v0", false, DEFAULT_READ_TIMEOUT);
+  }
+
+  public IPFSCompat(String host, int port, String version, boolean ssl, int readTimeout) {
+    this.host = host;
+    this.port = port;
+
+    if(ssl) {
+      this.protocol = "https";
+    } else {
+      this.protocol = "http";
+    }
+
+    this.version = version;
+    this.readTimeout = readTimeout;
+  }
+
+  public class DHT {
+    public List<String> findpeerListTimeout(Multihash id, int timeout, ExecutorService executor) {
+      BlockingQueue<CompletableFuture<Object>> results = new LinkedBlockingQueue<>();
+      executor.submit(() -> retrieveAndParseStream("dht/findpeer?arg=" + id, results));
+
+      try {
+        long stop = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(timeout);
+        while(System.currentTimeMillis() < stop) {
+          Map peer = (Map) results.poll(timeout, TimeUnit.SECONDS);
+          if ( peer != null ) {
+            if ( (int) peer.get("Type") == 2 ) {
+              return (List<String>)
+                  ((Map)
+                      ((List) peer.get("Responses")
+                      ).get(0)
+                  ).get("Addrs");
+            }
+            //else: response contains no Addrs, so ignore it.

Review comment:
       Please remove commented out code.

##########
File path: contrib/storage-ipfs/src/main/java/org/apache/drill/exec/store/ipfs/IPFSGroupScan.java
##########
@@ -0,0 +1,456 @@
+/*
+ * Copyright (c) 2018-2020 Bowen Ding, Yuedong Xu, Liang Wang
+ *
+ * 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.drill.exec.store.ipfs;
+
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.shaded.guava.com.google.common.cache.LoadingCache;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.collect.ArrayListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.ListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import io.ipfs.api.MerkleNode;
+import io.ipfs.multihash.Multihash;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.util.DrillVersionInfo;
+import org.apache.drill.exec.coord.ClusterCoordinator;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.schedule.AffinityCreator;
+import org.apache.drill.exec.store.schedule.AssignmentCreator;
+import org.apache.drill.exec.store.schedule.CompleteWork;
+import org.apache.drill.exec.store.schedule.EndpointByteMap;
+import org.apache.drill.exec.store.schedule.EndpointByteMapImpl;
+import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.RecursiveTask;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.drill.exec.store.ipfs.IPFSStoragePluginConfig.IPFSTimeOut.FETCH_DATA;
+
+@JsonTypeName("ipfs-scan")
+public class IPFSGroupScan extends AbstractGroupScan {
+  private static final Logger logger = LoggerFactory.getLogger(IPFSGroupScan.class);
+  private IPFSContext ipfsContext;
+  private IPFSScanSpec ipfsScanSpec;
+  private IPFSStoragePluginConfig config;
+  private List<SchemaPath> columns;
+
+  private static long DEFAULT_NODE_SIZE = 1000l;
+
+  private ListMultimap<Integer, IPFSWork> assignments;
+  private List<IPFSWork> ipfsWorkList = Lists.newArrayList();
+  private Map<String, List<IPFSWork>> endpointWorksMap;
+  private List<EndpointAffinity> affinities;
+
+  @JsonCreator
+  public IPFSGroupScan(@JsonProperty("IPFSScanSpec") IPFSScanSpec ipfsScanSpec,
+                       @JsonProperty("IPFSStoragePluginConfig") IPFSStoragePluginConfig ipfsStoragePluginConfig,
+                       @JsonProperty("columns") List<SchemaPath> columns,
+                       @JacksonInject StoragePluginRegistry pluginRegistry) throws IOException, ExecutionSetupException {
+    this(
+        ((IPFSStoragePlugin) pluginRegistry.getPlugin(ipfsStoragePluginConfig)).getIPFSContext(),
+        ipfsScanSpec,
+        columns
+    );
+  }
+
+  public IPFSGroupScan(IPFSContext ipfsContext,
+                       IPFSScanSpec ipfsScanSpec,
+                       List<SchemaPath> columns) {
+    super((String) null);
+    this.ipfsContext = ipfsContext;
+    this.ipfsScanSpec = ipfsScanSpec;
+    this.config = ipfsContext.getStoragePluginConfig();
+    logger.debug("GroupScan constructor called with columns {}", columns);
+    this.columns = columns == null || columns.size() == 0? ALL_COLUMNS : columns;
+    init();
+  }
+
+  private void init() {
+    IPFSHelper ipfsHelper = ipfsContext.getIPFSHelper();
+    ipfsHelper.setMaxPeersPerLeaf(config.getMaxNodesPerLeaf());
+    ipfsHelper.setTimeouts(config.getIpfsTimeouts());
+    endpointWorksMap = new HashMap<>();
+
+    Multihash topHash = ipfsScanSpec.getTargetHash(ipfsHelper);
+    LoadingCache<Multihash, IPFSPeer> peerMap = ipfsContext.getIPFSPeerCache();
+
+    try {
+      //TODO detect and warn about loops/recursions in a malformed tree
+      class IPFSTreeFlattener extends RecursiveTask<Map<Multihash, String>> {
+        private Multihash hash;
+        private boolean isProvider;
+        private Map<Multihash, String> ret = new LinkedHashMap<>();
+
+        public IPFSTreeFlattener(Multihash hash, boolean isProvider) {
+          this.hash = hash;
+          this.isProvider = isProvider;
+        }
+
+        @Override
+        public Map<Multihash, String> compute() {
+          try {
+            if (isProvider) {
+              IPFSPeer peer = peerMap.getUnchecked(hash);
+              ret.put(hash, peer.hasDrillbitAddress() ? peer.getDrillbitAddress().get() : null);
+              return ret;
+            }
+
+            MerkleNode metaOrSimpleNode = ipfsHelper.timedFailure(ipfsHelper.getClient().object::links, hash, config.getIpfsTimeout(FETCH_DATA));
+            if (metaOrSimpleNode.links.size() > 0) {
+              logger.debug("{} is a meta node", hash);
+              //TODO do something useful with leaf size, e.g. hint Drill about operation costs
+              List<Multihash> intermediates = metaOrSimpleNode.links.stream().map(x -> x.hash).collect(Collectors.toList());
+
+              ImmutableList.Builder<IPFSTreeFlattener> builder = ImmutableList.builder();
+              for (Multihash intermediate : intermediates.subList(1, intermediates.size())) {
+                builder.add(new IPFSTreeFlattener(intermediate, false));
+              }
+              ImmutableList<IPFSTreeFlattener> subtasks = builder.build();
+              subtasks.forEach(IPFSTreeFlattener::fork);
+
+              IPFSTreeFlattener first = new IPFSTreeFlattener(intermediates.get(0), false);
+              ret.putAll(first.compute());
+              subtasks.reverse().forEach(
+                  subtask -> ret.putAll(subtask.join())
+              );
+
+            } else {
+              logger.debug("{} is a simple node", hash);
+              List<IPFSPeer> providers = ipfsHelper.findprovsTimeout(hash).stream()
+                  .map(id ->
+                    peerMap.getUnchecked(id)
+                  )
+                  .collect(Collectors.toList());
+              //FIXME isDrillReady may block threads
+              providers = providers.stream()
+                  .filter(IPFSPeer::isDrillReady)
+                  .collect(Collectors.toList());
+              if (providers.size() < 1) {
+                logger.warn("No drill-ready provider found for leaf {}, adding foreman as the provider", hash);
+                providers.add(ipfsContext.getMyself());
+              }
+
+              logger.debug("Got {} providers for {} from IPFS", providers.size(), hash);
+              ImmutableList.Builder<IPFSTreeFlattener> builder = ImmutableList.builder();
+              for (IPFSPeer provider : providers.subList(1, providers.size())) {
+                builder.add(new IPFSTreeFlattener(provider.getId(), true));
+              }
+              ImmutableList<IPFSTreeFlattener> subtasks = builder.build();
+              subtasks.forEach(IPFSTreeFlattener::fork);
+
+              List<String> possibleAddrs = new LinkedList<>();
+              Multihash firstProvider = providers.get(0).getId();
+              IPFSTreeFlattener firstTask = new IPFSTreeFlattener(firstProvider, true);
+              String firstAddr = firstTask.compute().get(firstProvider);
+              if (firstAddr != null) {
+                possibleAddrs.add(firstAddr);
+              }
+
+              subtasks.reverse().forEach(
+                  subtask -> {
+                    String addr = subtask.join().get(subtask.hash);
+                    if (addr != null) {
+                      possibleAddrs.add(addr);
+                    }
+                  }
+              );
+
+              if (possibleAddrs.size() < 1) {
+                logger.error("All attempts to find an appropriate provider address for {} have failed", hash);
+                throw new RuntimeException("No address found for any provider for leaf " + hash);
+              } else {
+                Random random = new Random();
+                String chosenAddr = possibleAddrs.get(random.nextInt(possibleAddrs.size()));
+                ret.clear();
+                ret.put(hash, chosenAddr);
+                logger.debug("Got peer host {} for leaf {}", chosenAddr, hash);
+              }
+            }
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+          return ret;
+        }
+      }
+
+      logger.debug("start to recursively expand nested IPFS hashes, topHash={}", topHash);
+
+      Stopwatch watch = Stopwatch.createStarted();
+      //FIXME parallelization width magic number, maybe a config entry?
+      ForkJoinPool forkJoinPool = new ForkJoinPool(config.getNumWorkerThreads());
+      IPFSTreeFlattener topTask = new IPFSTreeFlattener(topHash, false);
+      Map<Multihash, String> leafAddrMap = forkJoinPool.invoke(topTask);
+
+      logger.debug("Took {} ms to expand hash leaves", watch.elapsed(TimeUnit.MILLISECONDS));
+      logger.debug("Iterating on {} leaves...", leafAddrMap.size());
+      ClusterCoordinator coordinator = ipfsContext.getStoragePlugin().getContext().getClusterCoordinator();
+      for (Multihash leaf : leafAddrMap.keySet()) {
+        String peerHostname = leafAddrMap.get(leaf);
+
+        Optional<DrillbitEndpoint> oep = coordinator.getAvailableEndpoints()
+            .stream()
+            .filter(a -> a.getAddress().equals(peerHostname))
+            .findAny();
+        DrillbitEndpoint ep;
+        if (oep.isPresent()) {
+          ep = oep.get();
+          logger.debug("Using existing endpoint {}", ep.getAddress());
+        } else {
+          logger.debug("created new endpoint on the fly {}", peerHostname);
+          //TODO read ports & version info from IPFS instead of hard-coded
+          ep = DrillbitEndpoint.newBuilder()
+              .setAddress(peerHostname)
+              .setUserPort(31010)
+              .setControlPort(31011)
+              .setDataPort(31012)
+              .setHttpPort(8047)
+              .setVersion(DrillVersionInfo.getVersion())
+              .setState(DrillbitEndpoint.State.ONLINE)
+              .build();
+          //TODO how to safely remove endpoints that are no longer needed once the query is completed?
+          ClusterCoordinator.RegistrationHandle handle = coordinator.register(ep);
+        }
+
+        IPFSWork work = new IPFSWork(leaf.toBase58());
+        logger.debug("added endpoint {} to work {}", ep.getAddress(), work);
+        work.getByteMap().add(ep, DEFAULT_NODE_SIZE);
+        work.setOnEndpoint(ep);
+
+        if(endpointWorksMap.containsKey(ep.getAddress())) {
+          endpointWorksMap.get(ep.getAddress()).add(work);
+        } else {
+          List<IPFSWork> ipfsWorks = Lists.newArrayList();
+          ipfsWorks.add(work);
+          endpointWorksMap.put(ep.getAddress(), ipfsWorks);
+        }
+        ipfsWorkList.add(work);
+      }
+    }catch (Exception e) {
+      logger.debug("exception in init");
+      throw new RuntimeException(e);
+    }
+  }
+
+  private IPFSGroupScan(IPFSGroupScan that) {
+    super(that);
+    this.ipfsContext = that.ipfsContext;
+    this.ipfsScanSpec = that.ipfsScanSpec;
+    this.config = that.config;
+    this.assignments = that.assignments;
+    this.ipfsWorkList = that.ipfsWorkList;
+    this.endpointWorksMap = that.endpointWorksMap;
+    this.columns = that.columns;
+  }
+
+  @JsonProperty
+  public List<SchemaPath> getColumns() {
+    return columns;
+  }
+
+  @JsonIgnore
+  public IPFSStoragePlugin getStoragePlugin() {
+    return ipfsContext.getStoragePlugin();
+  }
+
+  @JsonProperty
+  public IPFSScanSpec getIPFSScanSpec() {
+    return ipfsScanSpec;
+  }
+
+  @Override
+  public List<EndpointAffinity> getOperatorAffinity() {
+    if (affinities == null) {
+      affinities = AffinityCreator.getAffinityMap(ipfsWorkList);
+    }
+    return affinities;
+  }
+
+  @Override
+  public int getMaxParallelizationWidth() {
+    DrillbitEndpoint myself = ipfsContext.getStoragePlugin().getContext().getEndpoint();
+    int width;
+    if (endpointWorksMap.containsKey(myself.getAddress())) {
+      // the foreman is also going to be a minor fragment worker under a UnionExchange operator
+      width = ipfsWorkList.size();
+    } else {
+      // the foreman does not hold data, so we have to force parallelization
+      // to make sure there is a UnionExchange operator
+      width = ipfsWorkList.size() + 1;
+    }
+    logger.debug("getMaxParallelizationWidth: {}", width);
+    return width;
+  }
+
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
+    logger.debug("ipfsWorkList.size() = {}", ipfsWorkList.size());
+    logger.debug("endpointWorksMap: {}", endpointWorksMap);
+    if (endpointWorksMap.size()>1) { //偶尔还会出错?
+      //incomingEndpoints是已经排好顺序的endpoints,和fragment 顺序对应

Review comment:
       Please remove or translate Chinese comments.

##########
File path: contrib/storage-ipfs/src/main/java/org/apache/drill/exec/store/ipfs/IPFSGroupScan.java
##########
@@ -0,0 +1,456 @@
+/*
+ * Copyright (c) 2018-2020 Bowen Ding, Yuedong Xu, Liang Wang
+ *
+ * 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.drill.exec.store.ipfs;
+
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.shaded.guava.com.google.common.cache.LoadingCache;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.collect.ArrayListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.ListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import io.ipfs.api.MerkleNode;
+import io.ipfs.multihash.Multihash;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.util.DrillVersionInfo;
+import org.apache.drill.exec.coord.ClusterCoordinator;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.schedule.AffinityCreator;
+import org.apache.drill.exec.store.schedule.AssignmentCreator;
+import org.apache.drill.exec.store.schedule.CompleteWork;
+import org.apache.drill.exec.store.schedule.EndpointByteMap;
+import org.apache.drill.exec.store.schedule.EndpointByteMapImpl;
+import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.RecursiveTask;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.drill.exec.store.ipfs.IPFSStoragePluginConfig.IPFSTimeOut.FETCH_DATA;
+
+@JsonTypeName("ipfs-scan")
+public class IPFSGroupScan extends AbstractGroupScan {
+  private static final Logger logger = LoggerFactory.getLogger(IPFSGroupScan.class);
+  private IPFSContext ipfsContext;
+  private IPFSScanSpec ipfsScanSpec;
+  private IPFSStoragePluginConfig config;
+  private List<SchemaPath> columns;
+
+  private static long DEFAULT_NODE_SIZE = 1000l;
+
+  private ListMultimap<Integer, IPFSWork> assignments;
+  private List<IPFSWork> ipfsWorkList = Lists.newArrayList();
+  private Map<String, List<IPFSWork>> endpointWorksMap;
+  private List<EndpointAffinity> affinities;
+
+  @JsonCreator
+  public IPFSGroupScan(@JsonProperty("IPFSScanSpec") IPFSScanSpec ipfsScanSpec,
+                       @JsonProperty("IPFSStoragePluginConfig") IPFSStoragePluginConfig ipfsStoragePluginConfig,
+                       @JsonProperty("columns") List<SchemaPath> columns,
+                       @JacksonInject StoragePluginRegistry pluginRegistry) throws IOException, ExecutionSetupException {
+    this(
+        ((IPFSStoragePlugin) pluginRegistry.getPlugin(ipfsStoragePluginConfig)).getIPFSContext(),
+        ipfsScanSpec,
+        columns
+    );
+  }
+
+  public IPFSGroupScan(IPFSContext ipfsContext,
+                       IPFSScanSpec ipfsScanSpec,
+                       List<SchemaPath> columns) {
+    super((String) null);
+    this.ipfsContext = ipfsContext;
+    this.ipfsScanSpec = ipfsScanSpec;
+    this.config = ipfsContext.getStoragePluginConfig();
+    logger.debug("GroupScan constructor called with columns {}", columns);
+    this.columns = columns == null || columns.size() == 0? ALL_COLUMNS : columns;
+    init();
+  }
+
+  private void init() {
+    IPFSHelper ipfsHelper = ipfsContext.getIPFSHelper();
+    ipfsHelper.setMaxPeersPerLeaf(config.getMaxNodesPerLeaf());
+    ipfsHelper.setTimeouts(config.getIpfsTimeouts());
+    endpointWorksMap = new HashMap<>();
+
+    Multihash topHash = ipfsScanSpec.getTargetHash(ipfsHelper);
+    LoadingCache<Multihash, IPFSPeer> peerMap = ipfsContext.getIPFSPeerCache();
+
+    try {
+      //TODO detect and warn about loops/recursions in a malformed tree
+      class IPFSTreeFlattener extends RecursiveTask<Map<Multihash, String>> {
+        private Multihash hash;
+        private boolean isProvider;
+        private Map<Multihash, String> ret = new LinkedHashMap<>();
+
+        public IPFSTreeFlattener(Multihash hash, boolean isProvider) {
+          this.hash = hash;
+          this.isProvider = isProvider;
+        }
+
+        @Override
+        public Map<Multihash, String> compute() {
+          try {
+            if (isProvider) {
+              IPFSPeer peer = peerMap.getUnchecked(hash);
+              ret.put(hash, peer.hasDrillbitAddress() ? peer.getDrillbitAddress().get() : null);
+              return ret;
+            }
+
+            MerkleNode metaOrSimpleNode = ipfsHelper.timedFailure(ipfsHelper.getClient().object::links, hash, config.getIpfsTimeout(FETCH_DATA));
+            if (metaOrSimpleNode.links.size() > 0) {
+              logger.debug("{} is a meta node", hash);
+              //TODO do something useful with leaf size, e.g. hint Drill about operation costs
+              List<Multihash> intermediates = metaOrSimpleNode.links.stream().map(x -> x.hash).collect(Collectors.toList());
+
+              ImmutableList.Builder<IPFSTreeFlattener> builder = ImmutableList.builder();
+              for (Multihash intermediate : intermediates.subList(1, intermediates.size())) {
+                builder.add(new IPFSTreeFlattener(intermediate, false));
+              }
+              ImmutableList<IPFSTreeFlattener> subtasks = builder.build();
+              subtasks.forEach(IPFSTreeFlattener::fork);
+
+              IPFSTreeFlattener first = new IPFSTreeFlattener(intermediates.get(0), false);
+              ret.putAll(first.compute());
+              subtasks.reverse().forEach(
+                  subtask -> ret.putAll(subtask.join())
+              );
+
+            } else {
+              logger.debug("{} is a simple node", hash);
+              List<IPFSPeer> providers = ipfsHelper.findprovsTimeout(hash).stream()
+                  .map(id ->
+                    peerMap.getUnchecked(id)
+                  )
+                  .collect(Collectors.toList());
+              //FIXME isDrillReady may block threads
+              providers = providers.stream()
+                  .filter(IPFSPeer::isDrillReady)
+                  .collect(Collectors.toList());
+              if (providers.size() < 1) {
+                logger.warn("No drill-ready provider found for leaf {}, adding foreman as the provider", hash);
+                providers.add(ipfsContext.getMyself());
+              }
+
+              logger.debug("Got {} providers for {} from IPFS", providers.size(), hash);
+              ImmutableList.Builder<IPFSTreeFlattener> builder = ImmutableList.builder();
+              for (IPFSPeer provider : providers.subList(1, providers.size())) {
+                builder.add(new IPFSTreeFlattener(provider.getId(), true));
+              }
+              ImmutableList<IPFSTreeFlattener> subtasks = builder.build();
+              subtasks.forEach(IPFSTreeFlattener::fork);
+
+              List<String> possibleAddrs = new LinkedList<>();
+              Multihash firstProvider = providers.get(0).getId();
+              IPFSTreeFlattener firstTask = new IPFSTreeFlattener(firstProvider, true);
+              String firstAddr = firstTask.compute().get(firstProvider);
+              if (firstAddr != null) {
+                possibleAddrs.add(firstAddr);
+              }
+
+              subtasks.reverse().forEach(
+                  subtask -> {
+                    String addr = subtask.join().get(subtask.hash);
+                    if (addr != null) {
+                      possibleAddrs.add(addr);
+                    }
+                  }
+              );
+
+              if (possibleAddrs.size() < 1) {
+                logger.error("All attempts to find an appropriate provider address for {} have failed", hash);
+                throw new RuntimeException("No address found for any provider for leaf " + hash);
+              } else {
+                Random random = new Random();
+                String chosenAddr = possibleAddrs.get(random.nextInt(possibleAddrs.size()));
+                ret.clear();
+                ret.put(hash, chosenAddr);
+                logger.debug("Got peer host {} for leaf {}", chosenAddr, hash);
+              }
+            }
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+          return ret;
+        }
+      }
+
+      logger.debug("start to recursively expand nested IPFS hashes, topHash={}", topHash);
+
+      Stopwatch watch = Stopwatch.createStarted();
+      //FIXME parallelization width magic number, maybe a config entry?
+      ForkJoinPool forkJoinPool = new ForkJoinPool(config.getNumWorkerThreads());
+      IPFSTreeFlattener topTask = new IPFSTreeFlattener(topHash, false);
+      Map<Multihash, String> leafAddrMap = forkJoinPool.invoke(topTask);
+
+      logger.debug("Took {} ms to expand hash leaves", watch.elapsed(TimeUnit.MILLISECONDS));
+      logger.debug("Iterating on {} leaves...", leafAddrMap.size());
+      ClusterCoordinator coordinator = ipfsContext.getStoragePlugin().getContext().getClusterCoordinator();
+      for (Multihash leaf : leafAddrMap.keySet()) {
+        String peerHostname = leafAddrMap.get(leaf);
+
+        Optional<DrillbitEndpoint> oep = coordinator.getAvailableEndpoints()
+            .stream()
+            .filter(a -> a.getAddress().equals(peerHostname))
+            .findAny();
+        DrillbitEndpoint ep;
+        if (oep.isPresent()) {
+          ep = oep.get();
+          logger.debug("Using existing endpoint {}", ep.getAddress());
+        } else {
+          logger.debug("created new endpoint on the fly {}", peerHostname);
+          //TODO read ports & version info from IPFS instead of hard-coded
+          ep = DrillbitEndpoint.newBuilder()
+              .setAddress(peerHostname)
+              .setUserPort(31010)
+              .setControlPort(31011)
+              .setDataPort(31012)
+              .setHttpPort(8047)
+              .setVersion(DrillVersionInfo.getVersion())
+              .setState(DrillbitEndpoint.State.ONLINE)
+              .build();
+          //TODO how to safely remove endpoints that are no longer needed once the query is completed?
+          ClusterCoordinator.RegistrationHandle handle = coordinator.register(ep);
+        }
+
+        IPFSWork work = new IPFSWork(leaf.toBase58());
+        logger.debug("added endpoint {} to work {}", ep.getAddress(), work);
+        work.getByteMap().add(ep, DEFAULT_NODE_SIZE);
+        work.setOnEndpoint(ep);
+
+        if(endpointWorksMap.containsKey(ep.getAddress())) {
+          endpointWorksMap.get(ep.getAddress()).add(work);
+        } else {
+          List<IPFSWork> ipfsWorks = Lists.newArrayList();
+          ipfsWorks.add(work);
+          endpointWorksMap.put(ep.getAddress(), ipfsWorks);
+        }
+        ipfsWorkList.add(work);
+      }
+    }catch (Exception e) {
+      logger.debug("exception in init");
+      throw new RuntimeException(e);
+    }
+  }
+
+  private IPFSGroupScan(IPFSGroupScan that) {
+    super(that);
+    this.ipfsContext = that.ipfsContext;
+    this.ipfsScanSpec = that.ipfsScanSpec;
+    this.config = that.config;
+    this.assignments = that.assignments;
+    this.ipfsWorkList = that.ipfsWorkList;
+    this.endpointWorksMap = that.endpointWorksMap;
+    this.columns = that.columns;
+  }
+
+  @JsonProperty
+  public List<SchemaPath> getColumns() {
+    return columns;
+  }
+
+  @JsonIgnore
+  public IPFSStoragePlugin getStoragePlugin() {
+    return ipfsContext.getStoragePlugin();
+  }
+
+  @JsonProperty
+  public IPFSScanSpec getIPFSScanSpec() {
+    return ipfsScanSpec;
+  }
+
+  @Override
+  public List<EndpointAffinity> getOperatorAffinity() {
+    if (affinities == null) {
+      affinities = AffinityCreator.getAffinityMap(ipfsWorkList);
+    }
+    return affinities;
+  }
+
+  @Override
+  public int getMaxParallelizationWidth() {
+    DrillbitEndpoint myself = ipfsContext.getStoragePlugin().getContext().getEndpoint();
+    int width;
+    if (endpointWorksMap.containsKey(myself.getAddress())) {
+      // the foreman is also going to be a minor fragment worker under a UnionExchange operator
+      width = ipfsWorkList.size();
+    } else {
+      // the foreman does not hold data, so we have to force parallelization
+      // to make sure there is a UnionExchange operator
+      width = ipfsWorkList.size() + 1;
+    }
+    logger.debug("getMaxParallelizationWidth: {}", width);
+    return width;
+  }
+
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
+    logger.debug("ipfsWorkList.size() = {}", ipfsWorkList.size());
+    logger.debug("endpointWorksMap: {}", endpointWorksMap);
+    if (endpointWorksMap.size()>1) { //偶尔还会出错?
+      //incomingEndpoints是已经排好顺序的endpoints,和fragment 顺序对应
+      logger.debug("Use manual assignment");
+      assignments = ArrayListMultimap.create();
+      for (int fragmentId = 0; fragmentId < incomingEndpoints.size(); fragmentId++) {
+        String address = incomingEndpoints.get(fragmentId).getAddress();
+        if (endpointWorksMap.containsKey(address)) { //如果对应的节点有工作
+          for (IPFSWork work : endpointWorksMap.get(address)) {
+            assignments.put(fragmentId, work);
+          }
+        } else //如果对应的节点没有工作安排,分配一个空work
+        {

Review comment:
       Does this need to be here?

##########
File path: contrib/storage-ipfs/src/main/java/org/apache/drill/exec/store/ipfs/IPFSCompat.java
##########
@@ -0,0 +1,202 @@
+/*
+ * Copyright (c) 2018-2020 Bowen Ding, Yuedong Xu, Liang Wang
+ *
+ * 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.drill.exec.store.ipfs;
+
+import io.ipfs.api.JSONParser;
+import io.ipfs.multihash.Multihash;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+/*
+ * Compatibility fixes for java-ipfs-http-client library
+ */
+public class IPFSCompat {
+  public final String host;
+  public final int port;
+  private final String version;
+  public final String protocol;
+  public final int readTimeout;
+  public static final int DEFAULT_READ_TIMEOUT = 0;
+
+  public final DHT dht = new DHT();
+  public final Name name = new Name();
+
+  public IPFSCompat(String host, int port) {
+    this(host, port, "/api/v0", false, DEFAULT_READ_TIMEOUT);
+  }
+
+  public IPFSCompat(String host, int port, String version, boolean ssl, int readTimeout) {
+    this.host = host;
+    this.port = port;
+
+    if(ssl) {
+      this.protocol = "https";
+    } else {
+      this.protocol = "http";
+    }
+
+    this.version = version;
+    this.readTimeout = readTimeout;
+  }
+
+  public class DHT {
+    public List<String> findpeerListTimeout(Multihash id, int timeout, ExecutorService executor) {

Review comment:
       As a request for code reviewers who are not experts in IPFS, would you please add some JavaDocs here and elsewhere so we can follow what's going on? 
   

##########
File path: contrib/storage-ipfs/src/main/java/org/apache/drill/exec/store/ipfs/IPFSContext.java
##########
@@ -0,0 +1,95 @@
+/*
+ * Copyright (c) 2018-2020 Bowen Ding, Yuedong Xu, Liang Wang

Review comment:
       @vvysotskyi 
   Can @dbw9580 put the authors names and perhaps contact info in the `README`?  Personally, I don't have a problem with that. 

##########
File path: contrib/storage-ipfs/src/main/java/org/apache/drill/exec/store/ipfs/IPFSGroupScan.java
##########
@@ -0,0 +1,456 @@
+/*
+ * Copyright (c) 2018-2020 Bowen Ding, Yuedong Xu, Liang Wang
+ *
+ * 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.drill.exec.store.ipfs;
+
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.shaded.guava.com.google.common.cache.LoadingCache;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.collect.ArrayListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.ListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import io.ipfs.api.MerkleNode;
+import io.ipfs.multihash.Multihash;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.util.DrillVersionInfo;
+import org.apache.drill.exec.coord.ClusterCoordinator;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.schedule.AffinityCreator;
+import org.apache.drill.exec.store.schedule.AssignmentCreator;
+import org.apache.drill.exec.store.schedule.CompleteWork;
+import org.apache.drill.exec.store.schedule.EndpointByteMap;
+import org.apache.drill.exec.store.schedule.EndpointByteMapImpl;
+import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.RecursiveTask;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.drill.exec.store.ipfs.IPFSStoragePluginConfig.IPFSTimeOut.FETCH_DATA;
+
+@JsonTypeName("ipfs-scan")
+public class IPFSGroupScan extends AbstractGroupScan {
+  private static final Logger logger = LoggerFactory.getLogger(IPFSGroupScan.class);
+  private IPFSContext ipfsContext;
+  private IPFSScanSpec ipfsScanSpec;
+  private IPFSStoragePluginConfig config;
+  private List<SchemaPath> columns;
+
+  private static long DEFAULT_NODE_SIZE = 1000l;
+
+  private ListMultimap<Integer, IPFSWork> assignments;
+  private List<IPFSWork> ipfsWorkList = Lists.newArrayList();
+  private Map<String, List<IPFSWork>> endpointWorksMap;
+  private List<EndpointAffinity> affinities;
+
+  @JsonCreator
+  public IPFSGroupScan(@JsonProperty("IPFSScanSpec") IPFSScanSpec ipfsScanSpec,
+                       @JsonProperty("IPFSStoragePluginConfig") IPFSStoragePluginConfig ipfsStoragePluginConfig,
+                       @JsonProperty("columns") List<SchemaPath> columns,
+                       @JacksonInject StoragePluginRegistry pluginRegistry) throws IOException, ExecutionSetupException {
+    this(
+        ((IPFSStoragePlugin) pluginRegistry.getPlugin(ipfsStoragePluginConfig)).getIPFSContext(),
+        ipfsScanSpec,
+        columns
+    );
+  }
+
+  public IPFSGroupScan(IPFSContext ipfsContext,
+                       IPFSScanSpec ipfsScanSpec,
+                       List<SchemaPath> columns) {
+    super((String) null);
+    this.ipfsContext = ipfsContext;
+    this.ipfsScanSpec = ipfsScanSpec;
+    this.config = ipfsContext.getStoragePluginConfig();
+    logger.debug("GroupScan constructor called with columns {}", columns);
+    this.columns = columns == null || columns.size() == 0? ALL_COLUMNS : columns;
+    init();
+  }
+
+  private void init() {
+    IPFSHelper ipfsHelper = ipfsContext.getIPFSHelper();
+    ipfsHelper.setMaxPeersPerLeaf(config.getMaxNodesPerLeaf());
+    ipfsHelper.setTimeouts(config.getIpfsTimeouts());
+    endpointWorksMap = new HashMap<>();
+
+    Multihash topHash = ipfsScanSpec.getTargetHash(ipfsHelper);
+    LoadingCache<Multihash, IPFSPeer> peerMap = ipfsContext.getIPFSPeerCache();
+
+    try {
+      //TODO detect and warn about loops/recursions in a malformed tree
+      class IPFSTreeFlattener extends RecursiveTask<Map<Multihash, String>> {
+        private Multihash hash;
+        private boolean isProvider;
+        private Map<Multihash, String> ret = new LinkedHashMap<>();
+
+        public IPFSTreeFlattener(Multihash hash, boolean isProvider) {
+          this.hash = hash;
+          this.isProvider = isProvider;
+        }
+
+        @Override
+        public Map<Multihash, String> compute() {
+          try {
+            if (isProvider) {
+              IPFSPeer peer = peerMap.getUnchecked(hash);
+              ret.put(hash, peer.hasDrillbitAddress() ? peer.getDrillbitAddress().get() : null);
+              return ret;
+            }
+
+            MerkleNode metaOrSimpleNode = ipfsHelper.timedFailure(ipfsHelper.getClient().object::links, hash, config.getIpfsTimeout(FETCH_DATA));
+            if (metaOrSimpleNode.links.size() > 0) {
+              logger.debug("{} is a meta node", hash);
+              //TODO do something useful with leaf size, e.g. hint Drill about operation costs
+              List<Multihash> intermediates = metaOrSimpleNode.links.stream().map(x -> x.hash).collect(Collectors.toList());
+
+              ImmutableList.Builder<IPFSTreeFlattener> builder = ImmutableList.builder();
+              for (Multihash intermediate : intermediates.subList(1, intermediates.size())) {
+                builder.add(new IPFSTreeFlattener(intermediate, false));
+              }
+              ImmutableList<IPFSTreeFlattener> subtasks = builder.build();
+              subtasks.forEach(IPFSTreeFlattener::fork);
+
+              IPFSTreeFlattener first = new IPFSTreeFlattener(intermediates.get(0), false);
+              ret.putAll(first.compute());
+              subtasks.reverse().forEach(
+                  subtask -> ret.putAll(subtask.join())
+              );
+
+            } else {
+              logger.debug("{} is a simple node", hash);
+              List<IPFSPeer> providers = ipfsHelper.findprovsTimeout(hash).stream()
+                  .map(id ->
+                    peerMap.getUnchecked(id)
+                  )
+                  .collect(Collectors.toList());
+              //FIXME isDrillReady may block threads

Review comment:
       ??

##########
File path: contrib/storage-ipfs/src/main/java/org/apache/drill/exec/store/ipfs/IPFSGroupScan.java
##########
@@ -0,0 +1,456 @@
+/*
+ * Copyright (c) 2018-2020 Bowen Ding, Yuedong Xu, Liang Wang
+ *
+ * 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.drill.exec.store.ipfs;
+
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.shaded.guava.com.google.common.cache.LoadingCache;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.collect.ArrayListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.ListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import io.ipfs.api.MerkleNode;
+import io.ipfs.multihash.Multihash;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.util.DrillVersionInfo;
+import org.apache.drill.exec.coord.ClusterCoordinator;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.schedule.AffinityCreator;
+import org.apache.drill.exec.store.schedule.AssignmentCreator;
+import org.apache.drill.exec.store.schedule.CompleteWork;
+import org.apache.drill.exec.store.schedule.EndpointByteMap;
+import org.apache.drill.exec.store.schedule.EndpointByteMapImpl;
+import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.RecursiveTask;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.drill.exec.store.ipfs.IPFSStoragePluginConfig.IPFSTimeOut.FETCH_DATA;
+
+@JsonTypeName("ipfs-scan")
+public class IPFSGroupScan extends AbstractGroupScan {
+  private static final Logger logger = LoggerFactory.getLogger(IPFSGroupScan.class);
+  private IPFSContext ipfsContext;
+  private IPFSScanSpec ipfsScanSpec;
+  private IPFSStoragePluginConfig config;
+  private List<SchemaPath> columns;
+
+  private static long DEFAULT_NODE_SIZE = 1000l;
+
+  private ListMultimap<Integer, IPFSWork> assignments;
+  private List<IPFSWork> ipfsWorkList = Lists.newArrayList();
+  private Map<String, List<IPFSWork>> endpointWorksMap;
+  private List<EndpointAffinity> affinities;
+
+  @JsonCreator
+  public IPFSGroupScan(@JsonProperty("IPFSScanSpec") IPFSScanSpec ipfsScanSpec,
+                       @JsonProperty("IPFSStoragePluginConfig") IPFSStoragePluginConfig ipfsStoragePluginConfig,
+                       @JsonProperty("columns") List<SchemaPath> columns,
+                       @JacksonInject StoragePluginRegistry pluginRegistry) throws IOException, ExecutionSetupException {
+    this(
+        ((IPFSStoragePlugin) pluginRegistry.getPlugin(ipfsStoragePluginConfig)).getIPFSContext(),
+        ipfsScanSpec,
+        columns
+    );
+  }
+
+  public IPFSGroupScan(IPFSContext ipfsContext,
+                       IPFSScanSpec ipfsScanSpec,
+                       List<SchemaPath> columns) {
+    super((String) null);
+    this.ipfsContext = ipfsContext;
+    this.ipfsScanSpec = ipfsScanSpec;
+    this.config = ipfsContext.getStoragePluginConfig();
+    logger.debug("GroupScan constructor called with columns {}", columns);
+    this.columns = columns == null || columns.size() == 0? ALL_COLUMNS : columns;
+    init();
+  }
+
+  private void init() {
+    IPFSHelper ipfsHelper = ipfsContext.getIPFSHelper();
+    ipfsHelper.setMaxPeersPerLeaf(config.getMaxNodesPerLeaf());
+    ipfsHelper.setTimeouts(config.getIpfsTimeouts());
+    endpointWorksMap = new HashMap<>();
+
+    Multihash topHash = ipfsScanSpec.getTargetHash(ipfsHelper);
+    LoadingCache<Multihash, IPFSPeer> peerMap = ipfsContext.getIPFSPeerCache();
+
+    try {
+      //TODO detect and warn about loops/recursions in a malformed tree
+      class IPFSTreeFlattener extends RecursiveTask<Map<Multihash, String>> {
+        private Multihash hash;
+        private boolean isProvider;
+        private Map<Multihash, String> ret = new LinkedHashMap<>();
+
+        public IPFSTreeFlattener(Multihash hash, boolean isProvider) {
+          this.hash = hash;
+          this.isProvider = isProvider;
+        }
+
+        @Override
+        public Map<Multihash, String> compute() {
+          try {
+            if (isProvider) {
+              IPFSPeer peer = peerMap.getUnchecked(hash);
+              ret.put(hash, peer.hasDrillbitAddress() ? peer.getDrillbitAddress().get() : null);
+              return ret;
+            }
+
+            MerkleNode metaOrSimpleNode = ipfsHelper.timedFailure(ipfsHelper.getClient().object::links, hash, config.getIpfsTimeout(FETCH_DATA));
+            if (metaOrSimpleNode.links.size() > 0) {
+              logger.debug("{} is a meta node", hash);
+              //TODO do something useful with leaf size, e.g. hint Drill about operation costs
+              List<Multihash> intermediates = metaOrSimpleNode.links.stream().map(x -> x.hash).collect(Collectors.toList());
+
+              ImmutableList.Builder<IPFSTreeFlattener> builder = ImmutableList.builder();
+              for (Multihash intermediate : intermediates.subList(1, intermediates.size())) {
+                builder.add(new IPFSTreeFlattener(intermediate, false));
+              }
+              ImmutableList<IPFSTreeFlattener> subtasks = builder.build();
+              subtasks.forEach(IPFSTreeFlattener::fork);
+
+              IPFSTreeFlattener first = new IPFSTreeFlattener(intermediates.get(0), false);
+              ret.putAll(first.compute());
+              subtasks.reverse().forEach(
+                  subtask -> ret.putAll(subtask.join())
+              );
+
+            } else {
+              logger.debug("{} is a simple node", hash);
+              List<IPFSPeer> providers = ipfsHelper.findprovsTimeout(hash).stream()
+                  .map(id ->
+                    peerMap.getUnchecked(id)
+                  )
+                  .collect(Collectors.toList());
+              //FIXME isDrillReady may block threads
+              providers = providers.stream()
+                  .filter(IPFSPeer::isDrillReady)
+                  .collect(Collectors.toList());
+              if (providers.size() < 1) {
+                logger.warn("No drill-ready provider found for leaf {}, adding foreman as the provider", hash);
+                providers.add(ipfsContext.getMyself());
+              }
+
+              logger.debug("Got {} providers for {} from IPFS", providers.size(), hash);
+              ImmutableList.Builder<IPFSTreeFlattener> builder = ImmutableList.builder();
+              for (IPFSPeer provider : providers.subList(1, providers.size())) {
+                builder.add(new IPFSTreeFlattener(provider.getId(), true));
+              }
+              ImmutableList<IPFSTreeFlattener> subtasks = builder.build();
+              subtasks.forEach(IPFSTreeFlattener::fork);
+
+              List<String> possibleAddrs = new LinkedList<>();
+              Multihash firstProvider = providers.get(0).getId();
+              IPFSTreeFlattener firstTask = new IPFSTreeFlattener(firstProvider, true);
+              String firstAddr = firstTask.compute().get(firstProvider);
+              if (firstAddr != null) {
+                possibleAddrs.add(firstAddr);
+              }
+
+              subtasks.reverse().forEach(
+                  subtask -> {
+                    String addr = subtask.join().get(subtask.hash);
+                    if (addr != null) {
+                      possibleAddrs.add(addr);
+                    }
+                  }
+              );
+
+              if (possibleAddrs.size() < 1) {
+                logger.error("All attempts to find an appropriate provider address for {} have failed", hash);
+                throw new RuntimeException("No address found for any provider for leaf " + hash);
+              } else {
+                Random random = new Random();
+                String chosenAddr = possibleAddrs.get(random.nextInt(possibleAddrs.size()));
+                ret.clear();
+                ret.put(hash, chosenAddr);
+                logger.debug("Got peer host {} for leaf {}", chosenAddr, hash);
+              }
+            }
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+          return ret;
+        }
+      }
+
+      logger.debug("start to recursively expand nested IPFS hashes, topHash={}", topHash);
+
+      Stopwatch watch = Stopwatch.createStarted();
+      //FIXME parallelization width magic number, maybe a config entry?
+      ForkJoinPool forkJoinPool = new ForkJoinPool(config.getNumWorkerThreads());
+      IPFSTreeFlattener topTask = new IPFSTreeFlattener(topHash, false);
+      Map<Multihash, String> leafAddrMap = forkJoinPool.invoke(topTask);
+
+      logger.debug("Took {} ms to expand hash leaves", watch.elapsed(TimeUnit.MILLISECONDS));
+      logger.debug("Iterating on {} leaves...", leafAddrMap.size());
+      ClusterCoordinator coordinator = ipfsContext.getStoragePlugin().getContext().getClusterCoordinator();
+      for (Multihash leaf : leafAddrMap.keySet()) {
+        String peerHostname = leafAddrMap.get(leaf);
+
+        Optional<DrillbitEndpoint> oep = coordinator.getAvailableEndpoints()
+            .stream()
+            .filter(a -> a.getAddress().equals(peerHostname))
+            .findAny();
+        DrillbitEndpoint ep;
+        if (oep.isPresent()) {
+          ep = oep.get();
+          logger.debug("Using existing endpoint {}", ep.getAddress());
+        } else {
+          logger.debug("created new endpoint on the fly {}", peerHostname);
+          //TODO read ports & version info from IPFS instead of hard-coded
+          ep = DrillbitEndpoint.newBuilder()
+              .setAddress(peerHostname)
+              .setUserPort(31010)
+              .setControlPort(31011)
+              .setDataPort(31012)
+              .setHttpPort(8047)
+              .setVersion(DrillVersionInfo.getVersion())
+              .setState(DrillbitEndpoint.State.ONLINE)
+              .build();
+          //TODO how to safely remove endpoints that are no longer needed once the query is completed?
+          ClusterCoordinator.RegistrationHandle handle = coordinator.register(ep);
+        }
+
+        IPFSWork work = new IPFSWork(leaf.toBase58());
+        logger.debug("added endpoint {} to work {}", ep.getAddress(), work);
+        work.getByteMap().add(ep, DEFAULT_NODE_SIZE);
+        work.setOnEndpoint(ep);
+
+        if(endpointWorksMap.containsKey(ep.getAddress())) {
+          endpointWorksMap.get(ep.getAddress()).add(work);
+        } else {
+          List<IPFSWork> ipfsWorks = Lists.newArrayList();
+          ipfsWorks.add(work);
+          endpointWorksMap.put(ep.getAddress(), ipfsWorks);
+        }
+        ipfsWorkList.add(work);
+      }
+    }catch (Exception e) {
+      logger.debug("exception in init");
+      throw new RuntimeException(e);
+    }
+  }
+
+  private IPFSGroupScan(IPFSGroupScan that) {
+    super(that);
+    this.ipfsContext = that.ipfsContext;
+    this.ipfsScanSpec = that.ipfsScanSpec;
+    this.config = that.config;
+    this.assignments = that.assignments;
+    this.ipfsWorkList = that.ipfsWorkList;
+    this.endpointWorksMap = that.endpointWorksMap;
+    this.columns = that.columns;
+  }
+
+  @JsonProperty
+  public List<SchemaPath> getColumns() {
+    return columns;
+  }
+
+  @JsonIgnore
+  public IPFSStoragePlugin getStoragePlugin() {
+    return ipfsContext.getStoragePlugin();
+  }
+
+  @JsonProperty
+  public IPFSScanSpec getIPFSScanSpec() {
+    return ipfsScanSpec;
+  }
+
+  @Override
+  public List<EndpointAffinity> getOperatorAffinity() {
+    if (affinities == null) {
+      affinities = AffinityCreator.getAffinityMap(ipfsWorkList);
+    }
+    return affinities;
+  }
+
+  @Override
+  public int getMaxParallelizationWidth() {
+    DrillbitEndpoint myself = ipfsContext.getStoragePlugin().getContext().getEndpoint();
+    int width;
+    if (endpointWorksMap.containsKey(myself.getAddress())) {
+      // the foreman is also going to be a minor fragment worker under a UnionExchange operator
+      width = ipfsWorkList.size();
+    } else {
+      // the foreman does not hold data, so we have to force parallelization
+      // to make sure there is a UnionExchange operator
+      width = ipfsWorkList.size() + 1;
+    }
+    logger.debug("getMaxParallelizationWidth: {}", width);
+    return width;
+  }
+
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
+    logger.debug("ipfsWorkList.size() = {}", ipfsWorkList.size());
+    logger.debug("endpointWorksMap: {}", endpointWorksMap);
+    if (endpointWorksMap.size()>1) { //偶尔还会出错?
+      //incomingEndpoints是已经排好顺序的endpoints,和fragment 顺序对应
+      logger.debug("Use manual assignment");
+      assignments = ArrayListMultimap.create();
+      for (int fragmentId = 0; fragmentId < incomingEndpoints.size(); fragmentId++) {
+        String address = incomingEndpoints.get(fragmentId).getAddress();
+        if (endpointWorksMap.containsKey(address)) { //如果对应的节点有工作
+          for (IPFSWork work : endpointWorksMap.get(address)) {
+            assignments.put(fragmentId, work);
+          }
+        } else //如果对应的节点没有工作安排,分配一个空work
+        {
+
+        }
+      }
+    }
+    else //如果出问题,按照系统默认分配模式?
+    {
+     logger.debug("Use AssignmentCreator");
+      assignments = AssignmentCreator.getMappings(incomingEndpoints, ipfsWorkList);
+    }
+
+    for (int i = 0; i < incomingEndpoints.size(); i++) {
+      logger.debug("Fragment {} on endpoint {} is assigned with works: {}", i, incomingEndpoints.get(i).getAddress(), assignments.get(i));
+    }
+  }
+
+  @Override
+  public IPFSSubScan getSpecificScan(int minorFragmentId) {
+    logger.debug(String.format("getSpecificScan: minorFragmentId = %d", minorFragmentId));
+    List<IPFSWork> workList = assignments.get(minorFragmentId);
+    logger.debug("workList == null: " + (workList == null? "true": "false"));
+    logger.debug(String.format("workList.size(): %d", workList.size()));
+
+    List<Multihash> scanSpecList = Lists.newArrayList();
+
+    for (IPFSWork work : workList) {
+      scanSpecList.add(work.getPartialRootHash());
+    }
+
+    return new IPFSSubScan(ipfsContext, scanSpecList, ipfsScanSpec.getFormatExtension(), columns);
+  }
+
+  @Override
+  public ScanStats getScanStats() {
+    //FIXME why 100000 * size?
+    long recordCount = 100000 * endpointWorksMap.size();

Review comment:
       This is actually really important.  Calcite uses a cost-based planner.  If you are pushing down filters, aggregates etc. you want to make sure that the scan stats go down as you push down the filters.  There's no penalty for setting this to an arbitrary large money to start with. 

##########
File path: contrib/storage-ipfs/src/main/java/org/apache/drill/exec/store/ipfs/IPFSScanBatchCreator.java
##########
@@ -0,0 +1,114 @@
+/*
+ * Copyright (c) 2018-2020 Bowen Ding, Yuedong Xu, Liang Wang
+ *
+ * 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.drill.exec.store.ipfs;
+
+import org.apache.drill.common.exceptions.ChildErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedScanFramework.ReaderFactory;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedScanFramework.ScanFrameworkBuilder;
+import org.apache.drill.exec.physical.impl.scan.framework.SchemaNegotiator;
+import org.apache.drill.exec.record.CloseableRecordBatch;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import io.ipfs.multihash.Multihash;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.ExecutorFragmentContext;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.record.RecordBatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public class IPFSScanBatchCreator implements BatchCreator<IPFSSubScan> {
+  private static final Logger logger = LoggerFactory.getLogger(IPFSScanBatchCreator.class);
+
+  @Override
+  public CloseableRecordBatch getBatch(ExecutorFragmentContext context, IPFSSubScan subScan, List<RecordBatch> children)
+      throws ExecutionSetupException {
+    Preconditions.checkArgument(children.isEmpty());
+    logger.debug(String.format("subScanSpecList.size = %d", subScan.getIPFSSubScanSpecList().size()));
+
+    try {
+      ScanFrameworkBuilder builder = createBuilder(context.getOptions(), subScan);
+      return builder.buildScanOperator(context, subScan);
+    } catch (UserException e) {
+      // Rethrow user exceptions directly

Review comment:
       This can be consolidated and just throw a `UserException`

##########
File path: contrib/storage-ipfs/src/test/java/org/apache/drill/exec/store/ipfs/IPFSTestBase.java
##########
@@ -0,0 +1,61 @@
+/*
+ * Copyright (c) 2018-2020 Bowen Ding, Yuedong Xu, Liang Wang
+ *
+ * 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.drill.exec.store.ipfs;
+
+

Review comment:
       Nit:  Extra lines

##########
File path: contrib/storage-ipfs/src/main/java/org/apache/drill/exec/store/ipfs/IPFSGroupScan.java
##########
@@ -0,0 +1,456 @@
+/*
+ * Copyright (c) 2018-2020 Bowen Ding, Yuedong Xu, Liang Wang
+ *
+ * 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.drill.exec.store.ipfs;
+
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.shaded.guava.com.google.common.cache.LoadingCache;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.collect.ArrayListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.ListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import io.ipfs.api.MerkleNode;
+import io.ipfs.multihash.Multihash;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.util.DrillVersionInfo;
+import org.apache.drill.exec.coord.ClusterCoordinator;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.schedule.AffinityCreator;
+import org.apache.drill.exec.store.schedule.AssignmentCreator;
+import org.apache.drill.exec.store.schedule.CompleteWork;
+import org.apache.drill.exec.store.schedule.EndpointByteMap;
+import org.apache.drill.exec.store.schedule.EndpointByteMapImpl;
+import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.RecursiveTask;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.drill.exec.store.ipfs.IPFSStoragePluginConfig.IPFSTimeOut.FETCH_DATA;
+
+@JsonTypeName("ipfs-scan")
+public class IPFSGroupScan extends AbstractGroupScan {
+  private static final Logger logger = LoggerFactory.getLogger(IPFSGroupScan.class);
+  private IPFSContext ipfsContext;
+  private IPFSScanSpec ipfsScanSpec;
+  private IPFSStoragePluginConfig config;
+  private List<SchemaPath> columns;
+
+  private static long DEFAULT_NODE_SIZE = 1000l;
+
+  private ListMultimap<Integer, IPFSWork> assignments;
+  private List<IPFSWork> ipfsWorkList = Lists.newArrayList();
+  private Map<String, List<IPFSWork>> endpointWorksMap;
+  private List<EndpointAffinity> affinities;
+
+  @JsonCreator
+  public IPFSGroupScan(@JsonProperty("IPFSScanSpec") IPFSScanSpec ipfsScanSpec,
+                       @JsonProperty("IPFSStoragePluginConfig") IPFSStoragePluginConfig ipfsStoragePluginConfig,
+                       @JsonProperty("columns") List<SchemaPath> columns,
+                       @JacksonInject StoragePluginRegistry pluginRegistry) throws IOException, ExecutionSetupException {
+    this(
+        ((IPFSStoragePlugin) pluginRegistry.getPlugin(ipfsStoragePluginConfig)).getIPFSContext(),
+        ipfsScanSpec,
+        columns
+    );
+  }
+
+  public IPFSGroupScan(IPFSContext ipfsContext,
+                       IPFSScanSpec ipfsScanSpec,
+                       List<SchemaPath> columns) {
+    super((String) null);
+    this.ipfsContext = ipfsContext;
+    this.ipfsScanSpec = ipfsScanSpec;
+    this.config = ipfsContext.getStoragePluginConfig();
+    logger.debug("GroupScan constructor called with columns {}", columns);
+    this.columns = columns == null || columns.size() == 0? ALL_COLUMNS : columns;
+    init();
+  }
+
+  private void init() {
+    IPFSHelper ipfsHelper = ipfsContext.getIPFSHelper();
+    ipfsHelper.setMaxPeersPerLeaf(config.getMaxNodesPerLeaf());
+    ipfsHelper.setTimeouts(config.getIpfsTimeouts());
+    endpointWorksMap = new HashMap<>();
+
+    Multihash topHash = ipfsScanSpec.getTargetHash(ipfsHelper);
+    LoadingCache<Multihash, IPFSPeer> peerMap = ipfsContext.getIPFSPeerCache();
+
+    try {
+      //TODO detect and warn about loops/recursions in a malformed tree
+      class IPFSTreeFlattener extends RecursiveTask<Map<Multihash, String>> {
+        private Multihash hash;
+        private boolean isProvider;
+        private Map<Multihash, String> ret = new LinkedHashMap<>();
+
+        public IPFSTreeFlattener(Multihash hash, boolean isProvider) {
+          this.hash = hash;
+          this.isProvider = isProvider;
+        }
+
+        @Override
+        public Map<Multihash, String> compute() {
+          try {
+            if (isProvider) {
+              IPFSPeer peer = peerMap.getUnchecked(hash);
+              ret.put(hash, peer.hasDrillbitAddress() ? peer.getDrillbitAddress().get() : null);
+              return ret;
+            }
+
+            MerkleNode metaOrSimpleNode = ipfsHelper.timedFailure(ipfsHelper.getClient().object::links, hash, config.getIpfsTimeout(FETCH_DATA));
+            if (metaOrSimpleNode.links.size() > 0) {
+              logger.debug("{} is a meta node", hash);
+              //TODO do something useful with leaf size, e.g. hint Drill about operation costs
+              List<Multihash> intermediates = metaOrSimpleNode.links.stream().map(x -> x.hash).collect(Collectors.toList());
+
+              ImmutableList.Builder<IPFSTreeFlattener> builder = ImmutableList.builder();
+              for (Multihash intermediate : intermediates.subList(1, intermediates.size())) {
+                builder.add(new IPFSTreeFlattener(intermediate, false));
+              }
+              ImmutableList<IPFSTreeFlattener> subtasks = builder.build();
+              subtasks.forEach(IPFSTreeFlattener::fork);
+
+              IPFSTreeFlattener first = new IPFSTreeFlattener(intermediates.get(0), false);
+              ret.putAll(first.compute());
+              subtasks.reverse().forEach(
+                  subtask -> ret.putAll(subtask.join())
+              );
+
+            } else {
+              logger.debug("{} is a simple node", hash);
+              List<IPFSPeer> providers = ipfsHelper.findprovsTimeout(hash).stream()
+                  .map(id ->
+                    peerMap.getUnchecked(id)
+                  )
+                  .collect(Collectors.toList());
+              //FIXME isDrillReady may block threads
+              providers = providers.stream()
+                  .filter(IPFSPeer::isDrillReady)
+                  .collect(Collectors.toList());
+              if (providers.size() < 1) {
+                logger.warn("No drill-ready provider found for leaf {}, adding foreman as the provider", hash);
+                providers.add(ipfsContext.getMyself());
+              }
+
+              logger.debug("Got {} providers for {} from IPFS", providers.size(), hash);
+              ImmutableList.Builder<IPFSTreeFlattener> builder = ImmutableList.builder();
+              for (IPFSPeer provider : providers.subList(1, providers.size())) {
+                builder.add(new IPFSTreeFlattener(provider.getId(), true));
+              }
+              ImmutableList<IPFSTreeFlattener> subtasks = builder.build();
+              subtasks.forEach(IPFSTreeFlattener::fork);
+
+              List<String> possibleAddrs = new LinkedList<>();
+              Multihash firstProvider = providers.get(0).getId();
+              IPFSTreeFlattener firstTask = new IPFSTreeFlattener(firstProvider, true);
+              String firstAddr = firstTask.compute().get(firstProvider);
+              if (firstAddr != null) {
+                possibleAddrs.add(firstAddr);
+              }
+
+              subtasks.reverse().forEach(
+                  subtask -> {
+                    String addr = subtask.join().get(subtask.hash);
+                    if (addr != null) {
+                      possibleAddrs.add(addr);
+                    }
+                  }
+              );
+
+              if (possibleAddrs.size() < 1) {
+                logger.error("All attempts to find an appropriate provider address for {} have failed", hash);
+                throw new RuntimeException("No address found for any provider for leaf " + hash);
+              } else {
+                Random random = new Random();
+                String chosenAddr = possibleAddrs.get(random.nextInt(possibleAddrs.size()));
+                ret.clear();
+                ret.put(hash, chosenAddr);
+                logger.debug("Got peer host {} for leaf {}", chosenAddr, hash);
+              }
+            }
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+          return ret;
+        }
+      }
+
+      logger.debug("start to recursively expand nested IPFS hashes, topHash={}", topHash);
+
+      Stopwatch watch = Stopwatch.createStarted();
+      //FIXME parallelization width magic number, maybe a config entry?
+      ForkJoinPool forkJoinPool = new ForkJoinPool(config.getNumWorkerThreads());
+      IPFSTreeFlattener topTask = new IPFSTreeFlattener(topHash, false);
+      Map<Multihash, String> leafAddrMap = forkJoinPool.invoke(topTask);
+
+      logger.debug("Took {} ms to expand hash leaves", watch.elapsed(TimeUnit.MILLISECONDS));
+      logger.debug("Iterating on {} leaves...", leafAddrMap.size());
+      ClusterCoordinator coordinator = ipfsContext.getStoragePlugin().getContext().getClusterCoordinator();
+      for (Multihash leaf : leafAddrMap.keySet()) {
+        String peerHostname = leafAddrMap.get(leaf);
+
+        Optional<DrillbitEndpoint> oep = coordinator.getAvailableEndpoints()
+            .stream()
+            .filter(a -> a.getAddress().equals(peerHostname))
+            .findAny();
+        DrillbitEndpoint ep;
+        if (oep.isPresent()) {
+          ep = oep.get();
+          logger.debug("Using existing endpoint {}", ep.getAddress());
+        } else {
+          logger.debug("created new endpoint on the fly {}", peerHostname);
+          //TODO read ports & version info from IPFS instead of hard-coded
+          ep = DrillbitEndpoint.newBuilder()
+              .setAddress(peerHostname)
+              .setUserPort(31010)
+              .setControlPort(31011)
+              .setDataPort(31012)
+              .setHttpPort(8047)
+              .setVersion(DrillVersionInfo.getVersion())
+              .setState(DrillbitEndpoint.State.ONLINE)
+              .build();
+          //TODO how to safely remove endpoints that are no longer needed once the query is completed?
+          ClusterCoordinator.RegistrationHandle handle = coordinator.register(ep);
+        }
+
+        IPFSWork work = new IPFSWork(leaf.toBase58());
+        logger.debug("added endpoint {} to work {}", ep.getAddress(), work);
+        work.getByteMap().add(ep, DEFAULT_NODE_SIZE);
+        work.setOnEndpoint(ep);
+
+        if(endpointWorksMap.containsKey(ep.getAddress())) {
+          endpointWorksMap.get(ep.getAddress()).add(work);
+        } else {
+          List<IPFSWork> ipfsWorks = Lists.newArrayList();
+          ipfsWorks.add(work);
+          endpointWorksMap.put(ep.getAddress(), ipfsWorks);
+        }
+        ipfsWorkList.add(work);
+      }
+    }catch (Exception e) {
+      logger.debug("exception in init");
+      throw new RuntimeException(e);
+    }
+  }
+
+  private IPFSGroupScan(IPFSGroupScan that) {
+    super(that);
+    this.ipfsContext = that.ipfsContext;
+    this.ipfsScanSpec = that.ipfsScanSpec;
+    this.config = that.config;
+    this.assignments = that.assignments;
+    this.ipfsWorkList = that.ipfsWorkList;
+    this.endpointWorksMap = that.endpointWorksMap;
+    this.columns = that.columns;
+  }
+
+  @JsonProperty
+  public List<SchemaPath> getColumns() {
+    return columns;
+  }
+
+  @JsonIgnore
+  public IPFSStoragePlugin getStoragePlugin() {
+    return ipfsContext.getStoragePlugin();
+  }
+
+  @JsonProperty
+  public IPFSScanSpec getIPFSScanSpec() {
+    return ipfsScanSpec;
+  }
+
+  @Override
+  public List<EndpointAffinity> getOperatorAffinity() {
+    if (affinities == null) {
+      affinities = AffinityCreator.getAffinityMap(ipfsWorkList);
+    }
+    return affinities;
+  }
+
+  @Override
+  public int getMaxParallelizationWidth() {
+    DrillbitEndpoint myself = ipfsContext.getStoragePlugin().getContext().getEndpoint();
+    int width;
+    if (endpointWorksMap.containsKey(myself.getAddress())) {
+      // the foreman is also going to be a minor fragment worker under a UnionExchange operator
+      width = ipfsWorkList.size();
+    } else {
+      // the foreman does not hold data, so we have to force parallelization
+      // to make sure there is a UnionExchange operator
+      width = ipfsWorkList.size() + 1;
+    }
+    logger.debug("getMaxParallelizationWidth: {}", width);
+    return width;
+  }
+
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
+    logger.debug("ipfsWorkList.size() = {}", ipfsWorkList.size());
+    logger.debug("endpointWorksMap: {}", endpointWorksMap);
+    if (endpointWorksMap.size()>1) { //偶尔还会出错?
+      //incomingEndpoints是已经排好顺序的endpoints,和fragment 顺序对应
+      logger.debug("Use manual assignment");
+      assignments = ArrayListMultimap.create();
+      for (int fragmentId = 0; fragmentId < incomingEndpoints.size(); fragmentId++) {
+        String address = incomingEndpoints.get(fragmentId).getAddress();
+        if (endpointWorksMap.containsKey(address)) { //如果对应的节点有工作
+          for (IPFSWork work : endpointWorksMap.get(address)) {
+            assignments.put(fragmentId, work);
+          }
+        } else //如果对应的节点没有工作安排,分配一个空work
+        {
+
+        }
+      }
+    }
+    else //如果出问题,按照系统默认分配模式?
+    {
+     logger.debug("Use AssignmentCreator");
+      assignments = AssignmentCreator.getMappings(incomingEndpoints, ipfsWorkList);
+    }
+
+    for (int i = 0; i < incomingEndpoints.size(); i++) {
+      logger.debug("Fragment {} on endpoint {} is assigned with works: {}", i, incomingEndpoints.get(i).getAddress(), assignments.get(i));
+    }
+  }
+
+  @Override
+  public IPFSSubScan getSpecificScan(int minorFragmentId) {
+    logger.debug(String.format("getSpecificScan: minorFragmentId = %d", minorFragmentId));
+    List<IPFSWork> workList = assignments.get(minorFragmentId);
+    logger.debug("workList == null: " + (workList == null? "true": "false"));
+    logger.debug(String.format("workList.size(): %d", workList.size()));
+
+    List<Multihash> scanSpecList = Lists.newArrayList();
+
+    for (IPFSWork work : workList) {
+      scanSpecList.add(work.getPartialRootHash());
+    }
+
+    return new IPFSSubScan(ipfsContext, scanSpecList, ipfsScanSpec.getFormatExtension(), columns);
+  }
+
+  @Override
+  public ScanStats getScanStats() {
+    //FIXME why 100000 * size?
+    long recordCount = 100000 * endpointWorksMap.size();
+    return new ScanStats(ScanStats.GroupScanProperty.NO_EXACT_ROW_COUNT, recordCount, 1, recordCount);
+  }
+
+  @Override
+  public IPFSGroupScan clone(List<SchemaPath> columns){
+    logger.debug("IPFSGroupScan clone {}", columns);
+    IPFSGroupScan cloned = new IPFSGroupScan(this);
+    cloned.columns = columns;
+    return cloned;
+  }
+
+  @Override
+  @JsonIgnore
+  public boolean canPushdownProjects(List<SchemaPath> columns) {
+    //FIXME what does this mean?

Review comment:
       Remove comment.  This tells the planner that projection can be pushed down to the reader(s). 

##########
File path: contrib/storage-ipfs/src/main/java/org/apache/drill/exec/store/ipfs/IPFSGroupScan.java
##########
@@ -0,0 +1,456 @@
+/*
+ * Copyright (c) 2018-2020 Bowen Ding, Yuedong Xu, Liang Wang
+ *
+ * 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.drill.exec.store.ipfs;
+
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.shaded.guava.com.google.common.cache.LoadingCache;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.collect.ArrayListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.ListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import io.ipfs.api.MerkleNode;
+import io.ipfs.multihash.Multihash;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.util.DrillVersionInfo;
+import org.apache.drill.exec.coord.ClusterCoordinator;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.schedule.AffinityCreator;
+import org.apache.drill.exec.store.schedule.AssignmentCreator;
+import org.apache.drill.exec.store.schedule.CompleteWork;
+import org.apache.drill.exec.store.schedule.EndpointByteMap;
+import org.apache.drill.exec.store.schedule.EndpointByteMapImpl;
+import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.RecursiveTask;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.drill.exec.store.ipfs.IPFSStoragePluginConfig.IPFSTimeOut.FETCH_DATA;
+
+@JsonTypeName("ipfs-scan")
+public class IPFSGroupScan extends AbstractGroupScan {
+  private static final Logger logger = LoggerFactory.getLogger(IPFSGroupScan.class);
+  private IPFSContext ipfsContext;
+  private IPFSScanSpec ipfsScanSpec;
+  private IPFSStoragePluginConfig config;
+  private List<SchemaPath> columns;
+
+  private static long DEFAULT_NODE_SIZE = 1000l;
+
+  private ListMultimap<Integer, IPFSWork> assignments;
+  private List<IPFSWork> ipfsWorkList = Lists.newArrayList();
+  private Map<String, List<IPFSWork>> endpointWorksMap;
+  private List<EndpointAffinity> affinities;
+
+  @JsonCreator
+  public IPFSGroupScan(@JsonProperty("IPFSScanSpec") IPFSScanSpec ipfsScanSpec,
+                       @JsonProperty("IPFSStoragePluginConfig") IPFSStoragePluginConfig ipfsStoragePluginConfig,
+                       @JsonProperty("columns") List<SchemaPath> columns,
+                       @JacksonInject StoragePluginRegistry pluginRegistry) throws IOException, ExecutionSetupException {
+    this(
+        ((IPFSStoragePlugin) pluginRegistry.getPlugin(ipfsStoragePluginConfig)).getIPFSContext(),
+        ipfsScanSpec,
+        columns
+    );
+  }
+
+  public IPFSGroupScan(IPFSContext ipfsContext,
+                       IPFSScanSpec ipfsScanSpec,
+                       List<SchemaPath> columns) {
+    super((String) null);
+    this.ipfsContext = ipfsContext;
+    this.ipfsScanSpec = ipfsScanSpec;
+    this.config = ipfsContext.getStoragePluginConfig();
+    logger.debug("GroupScan constructor called with columns {}", columns);
+    this.columns = columns == null || columns.size() == 0? ALL_COLUMNS : columns;
+    init();
+  }
+
+  private void init() {
+    IPFSHelper ipfsHelper = ipfsContext.getIPFSHelper();
+    ipfsHelper.setMaxPeersPerLeaf(config.getMaxNodesPerLeaf());
+    ipfsHelper.setTimeouts(config.getIpfsTimeouts());
+    endpointWorksMap = new HashMap<>();
+
+    Multihash topHash = ipfsScanSpec.getTargetHash(ipfsHelper);
+    LoadingCache<Multihash, IPFSPeer> peerMap = ipfsContext.getIPFSPeerCache();
+
+    try {
+      //TODO detect and warn about loops/recursions in a malformed tree
+      class IPFSTreeFlattener extends RecursiveTask<Map<Multihash, String>> {
+        private Multihash hash;
+        private boolean isProvider;
+        private Map<Multihash, String> ret = new LinkedHashMap<>();
+
+        public IPFSTreeFlattener(Multihash hash, boolean isProvider) {
+          this.hash = hash;
+          this.isProvider = isProvider;
+        }
+
+        @Override
+        public Map<Multihash, String> compute() {
+          try {
+            if (isProvider) {
+              IPFSPeer peer = peerMap.getUnchecked(hash);
+              ret.put(hash, peer.hasDrillbitAddress() ? peer.getDrillbitAddress().get() : null);
+              return ret;
+            }
+
+            MerkleNode metaOrSimpleNode = ipfsHelper.timedFailure(ipfsHelper.getClient().object::links, hash, config.getIpfsTimeout(FETCH_DATA));
+            if (metaOrSimpleNode.links.size() > 0) {
+              logger.debug("{} is a meta node", hash);
+              //TODO do something useful with leaf size, e.g. hint Drill about operation costs
+              List<Multihash> intermediates = metaOrSimpleNode.links.stream().map(x -> x.hash).collect(Collectors.toList());
+
+              ImmutableList.Builder<IPFSTreeFlattener> builder = ImmutableList.builder();
+              for (Multihash intermediate : intermediates.subList(1, intermediates.size())) {
+                builder.add(new IPFSTreeFlattener(intermediate, false));
+              }
+              ImmutableList<IPFSTreeFlattener> subtasks = builder.build();
+              subtasks.forEach(IPFSTreeFlattener::fork);
+
+              IPFSTreeFlattener first = new IPFSTreeFlattener(intermediates.get(0), false);
+              ret.putAll(first.compute());
+              subtasks.reverse().forEach(
+                  subtask -> ret.putAll(subtask.join())
+              );
+
+            } else {
+              logger.debug("{} is a simple node", hash);
+              List<IPFSPeer> providers = ipfsHelper.findprovsTimeout(hash).stream()
+                  .map(id ->
+                    peerMap.getUnchecked(id)
+                  )
+                  .collect(Collectors.toList());
+              //FIXME isDrillReady may block threads
+              providers = providers.stream()
+                  .filter(IPFSPeer::isDrillReady)
+                  .collect(Collectors.toList());
+              if (providers.size() < 1) {
+                logger.warn("No drill-ready provider found for leaf {}, adding foreman as the provider", hash);
+                providers.add(ipfsContext.getMyself());
+              }
+
+              logger.debug("Got {} providers for {} from IPFS", providers.size(), hash);
+              ImmutableList.Builder<IPFSTreeFlattener> builder = ImmutableList.builder();
+              for (IPFSPeer provider : providers.subList(1, providers.size())) {
+                builder.add(new IPFSTreeFlattener(provider.getId(), true));
+              }
+              ImmutableList<IPFSTreeFlattener> subtasks = builder.build();
+              subtasks.forEach(IPFSTreeFlattener::fork);
+
+              List<String> possibleAddrs = new LinkedList<>();
+              Multihash firstProvider = providers.get(0).getId();
+              IPFSTreeFlattener firstTask = new IPFSTreeFlattener(firstProvider, true);
+              String firstAddr = firstTask.compute().get(firstProvider);
+              if (firstAddr != null) {
+                possibleAddrs.add(firstAddr);
+              }
+
+              subtasks.reverse().forEach(
+                  subtask -> {
+                    String addr = subtask.join().get(subtask.hash);
+                    if (addr != null) {
+                      possibleAddrs.add(addr);
+                    }
+                  }
+              );
+
+              if (possibleAddrs.size() < 1) {
+                logger.error("All attempts to find an appropriate provider address for {} have failed", hash);
+                throw new RuntimeException("No address found for any provider for leaf " + hash);
+              } else {
+                Random random = new Random();
+                String chosenAddr = possibleAddrs.get(random.nextInt(possibleAddrs.size()));
+                ret.clear();
+                ret.put(hash, chosenAddr);
+                logger.debug("Got peer host {} for leaf {}", chosenAddr, hash);
+              }
+            }
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+          return ret;
+        }
+      }
+
+      logger.debug("start to recursively expand nested IPFS hashes, topHash={}", topHash);
+
+      Stopwatch watch = Stopwatch.createStarted();
+      //FIXME parallelization width magic number, maybe a config entry?
+      ForkJoinPool forkJoinPool = new ForkJoinPool(config.getNumWorkerThreads());
+      IPFSTreeFlattener topTask = new IPFSTreeFlattener(topHash, false);
+      Map<Multihash, String> leafAddrMap = forkJoinPool.invoke(topTask);
+
+      logger.debug("Took {} ms to expand hash leaves", watch.elapsed(TimeUnit.MILLISECONDS));
+      logger.debug("Iterating on {} leaves...", leafAddrMap.size());
+      ClusterCoordinator coordinator = ipfsContext.getStoragePlugin().getContext().getClusterCoordinator();
+      for (Multihash leaf : leafAddrMap.keySet()) {
+        String peerHostname = leafAddrMap.get(leaf);
+
+        Optional<DrillbitEndpoint> oep = coordinator.getAvailableEndpoints()
+            .stream()
+            .filter(a -> a.getAddress().equals(peerHostname))
+            .findAny();
+        DrillbitEndpoint ep;
+        if (oep.isPresent()) {
+          ep = oep.get();
+          logger.debug("Using existing endpoint {}", ep.getAddress());
+        } else {
+          logger.debug("created new endpoint on the fly {}", peerHostname);
+          //TODO read ports & version info from IPFS instead of hard-coded
+          ep = DrillbitEndpoint.newBuilder()
+              .setAddress(peerHostname)
+              .setUserPort(31010)
+              .setControlPort(31011)
+              .setDataPort(31012)
+              .setHttpPort(8047)
+              .setVersion(DrillVersionInfo.getVersion())
+              .setState(DrillbitEndpoint.State.ONLINE)
+              .build();
+          //TODO how to safely remove endpoints that are no longer needed once the query is completed?
+          ClusterCoordinator.RegistrationHandle handle = coordinator.register(ep);
+        }
+
+        IPFSWork work = new IPFSWork(leaf.toBase58());
+        logger.debug("added endpoint {} to work {}", ep.getAddress(), work);
+        work.getByteMap().add(ep, DEFAULT_NODE_SIZE);
+        work.setOnEndpoint(ep);
+
+        if(endpointWorksMap.containsKey(ep.getAddress())) {
+          endpointWorksMap.get(ep.getAddress()).add(work);
+        } else {
+          List<IPFSWork> ipfsWorks = Lists.newArrayList();
+          ipfsWorks.add(work);
+          endpointWorksMap.put(ep.getAddress(), ipfsWorks);
+        }
+        ipfsWorkList.add(work);
+      }
+    }catch (Exception e) {
+      logger.debug("exception in init");
+      throw new RuntimeException(e);
+    }
+  }
+
+  private IPFSGroupScan(IPFSGroupScan that) {
+    super(that);
+    this.ipfsContext = that.ipfsContext;
+    this.ipfsScanSpec = that.ipfsScanSpec;
+    this.config = that.config;
+    this.assignments = that.assignments;
+    this.ipfsWorkList = that.ipfsWorkList;
+    this.endpointWorksMap = that.endpointWorksMap;
+    this.columns = that.columns;
+  }
+
+  @JsonProperty
+  public List<SchemaPath> getColumns() {
+    return columns;
+  }
+
+  @JsonIgnore
+  public IPFSStoragePlugin getStoragePlugin() {
+    return ipfsContext.getStoragePlugin();
+  }
+
+  @JsonProperty
+  public IPFSScanSpec getIPFSScanSpec() {
+    return ipfsScanSpec;
+  }
+
+  @Override
+  public List<EndpointAffinity> getOperatorAffinity() {
+    if (affinities == null) {
+      affinities = AffinityCreator.getAffinityMap(ipfsWorkList);
+    }
+    return affinities;
+  }
+
+  @Override
+  public int getMaxParallelizationWidth() {
+    DrillbitEndpoint myself = ipfsContext.getStoragePlugin().getContext().getEndpoint();
+    int width;
+    if (endpointWorksMap.containsKey(myself.getAddress())) {
+      // the foreman is also going to be a minor fragment worker under a UnionExchange operator
+      width = ipfsWorkList.size();
+    } else {
+      // the foreman does not hold data, so we have to force parallelization
+      // to make sure there is a UnionExchange operator
+      width = ipfsWorkList.size() + 1;
+    }
+    logger.debug("getMaxParallelizationWidth: {}", width);
+    return width;
+  }
+
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
+    logger.debug("ipfsWorkList.size() = {}", ipfsWorkList.size());
+    logger.debug("endpointWorksMap: {}", endpointWorksMap);
+    if (endpointWorksMap.size()>1) { //偶尔还会出错?
+      //incomingEndpoints是已经排好顺序的endpoints,和fragment 顺序对应
+      logger.debug("Use manual assignment");
+      assignments = ArrayListMultimap.create();
+      for (int fragmentId = 0; fragmentId < incomingEndpoints.size(); fragmentId++) {
+        String address = incomingEndpoints.get(fragmentId).getAddress();
+        if (endpointWorksMap.containsKey(address)) { //如果对应的节点有工作
+          for (IPFSWork work : endpointWorksMap.get(address)) {
+            assignments.put(fragmentId, work);
+          }
+        } else //如果对应的节点没有工作安排,分配一个空work
+        {
+
+        }
+      }
+    }
+    else //如果出问题,按照系统默认分配模式?
+    {
+     logger.debug("Use AssignmentCreator");
+      assignments = AssignmentCreator.getMappings(incomingEndpoints, ipfsWorkList);
+    }
+
+    for (int i = 0; i < incomingEndpoints.size(); i++) {
+      logger.debug("Fragment {} on endpoint {} is assigned with works: {}", i, incomingEndpoints.get(i).getAddress(), assignments.get(i));
+    }
+  }
+
+  @Override
+  public IPFSSubScan getSpecificScan(int minorFragmentId) {
+    logger.debug(String.format("getSpecificScan: minorFragmentId = %d", minorFragmentId));
+    List<IPFSWork> workList = assignments.get(minorFragmentId);
+    logger.debug("workList == null: " + (workList == null? "true": "false"));
+    logger.debug(String.format("workList.size(): %d", workList.size()));
+
+    List<Multihash> scanSpecList = Lists.newArrayList();
+
+    for (IPFSWork work : workList) {
+      scanSpecList.add(work.getPartialRootHash());
+    }
+
+    return new IPFSSubScan(ipfsContext, scanSpecList, ipfsScanSpec.getFormatExtension(), columns);
+  }
+
+  @Override
+  public ScanStats getScanStats() {
+    //FIXME why 100000 * size?
+    long recordCount = 100000 * endpointWorksMap.size();
+    return new ScanStats(ScanStats.GroupScanProperty.NO_EXACT_ROW_COUNT, recordCount, 1, recordCount);
+  }
+
+  @Override
+  public IPFSGroupScan clone(List<SchemaPath> columns){
+    logger.debug("IPFSGroupScan clone {}", columns);
+    IPFSGroupScan cloned = new IPFSGroupScan(this);
+    cloned.columns = columns;
+    return cloned;
+  }
+
+  @Override
+  @JsonIgnore
+  public boolean canPushdownProjects(List<SchemaPath> columns) {
+    //FIXME what does this mean?
+    return true;
+  }
+
+  @Override
+  @JsonIgnore
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.isEmpty());
+    logger.debug("getNewWithChildren called");
+    return new IPFSGroupScan(this);
+  }
+
+
+
+  @Override
+  public String getDigest() {
+    return toString();
+  }
+
+  @Override
+  public String toString() {

Review comment:
       Please use `PlanStringBuilder` here and in other classes which are serialized.  
   
   See below:
   https://github.com/apache/drill/blob/1b95c0a8cfce23e11596353a821a5216fd1a983d/contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/HttpGroupScan.java#L228-L235

##########
File path: contrib/storage-ipfs/src/main/java/org/apache/drill/exec/store/ipfs/IPFSSubScan.java
##########
@@ -0,0 +1,182 @@
+/*
+ * Copyright (c) 2018-2020 Bowen Ding, Yuedong Xu, Liang Wang
+ *
+ * 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.drill.exec.store.ipfs;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.JsonDeserializer;
+import com.fasterxml.jackson.databind.JsonSerializer;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
+import io.ipfs.multihash.Multihash;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.AbstractBase;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.SubScan;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+/*import org.apache.drill.common.expression.SchemaPath;*/
+
+@JsonTypeName("ipfs-sub-scan")
+public class IPFSSubScan extends AbstractBase implements SubScan {
+  private static int IPFS_SUB_SCAN_VALUE = 19155;
+  private final IPFSContext ipfsContext;
+  private final List<Multihash> ipfsSubScanSpecList;
+  private final IPFSScanSpec.Format format;
+  private final List<SchemaPath> columns;
+
+
+  @JsonCreator
+  public IPFSSubScan(@JacksonInject StoragePluginRegistry registry,
+                     @JsonProperty("IPFSStoragePluginConfig") IPFSStoragePluginConfig ipfsStoragePluginConfig,
+                     @JsonProperty("IPFSSubScanSpec") @JsonDeserialize(using=MultihashDeserializer.class) List<Multihash> ipfsSubScanSpecList,
+                     @JsonProperty("format") IPFSScanSpec.Format format,
+                     @JsonProperty("columns") List<SchemaPath> columns
+                     ) throws ExecutionSetupException {
+    super((String) null);
+    IPFSStoragePlugin plugin = (IPFSStoragePlugin) registry.getPlugin(ipfsStoragePluginConfig);
+    ipfsContext = plugin.getIPFSContext();
+    this.ipfsSubScanSpecList = ipfsSubScanSpecList;
+    this.format = format;
+    this.columns = columns;
+  }
+
+  public IPFSSubScan(IPFSContext ipfsContext, List<Multihash> ipfsSubScanSpecList, IPFSScanSpec.Format format, List<SchemaPath> columns) {
+    super((String) null);
+    this.ipfsContext = ipfsContext;
+    this.ipfsSubScanSpecList = ipfsSubScanSpecList;
+    this.format = format;
+    this.columns = columns;
+  }
+
+  @JsonIgnore
+  public IPFSContext getIPFSContext() {
+    return ipfsContext;
+  }
+
+  @JsonProperty("IPFSStoragePluginConfig")
+  public IPFSStoragePluginConfig getIPFSStoragePluginConfig() {
+    return ipfsContext.getStoragePluginConfig();
+  }
+
+  @JsonProperty("columns")
+  public List<SchemaPath> getColumns() {
+    return columns;
+  }
+
+  @JsonProperty("format")
+  public IPFSScanSpec.Format getFormat() {
+    return format;
+  }
+

Review comment:
       I think you might want to overwrite the `toString()` method in the SubScan. Again, use the `PlanStringBuilder` for this.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



Mime
View raw message