Github user hyunsik commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/58#discussion_r14805622
  
    --- Diff: tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java 
---
    @@ -96,29 +113,83 @@ public void testAdjustFetchProcess() {
       @Test
       public void testStatus() throws Exception {
         Random rnd = new Random();
    -    FileWriter writer = new FileWriter(INPUT_DIR + "data");
    -    String data;
    +    QueryId queryId = QueryIdFactory.NULL_QUERY_ID;
    +    String sid = "1";
    +    String ta = "1_0";
    +    String partId = "1";
    +
    +    String dataPath = INPUT_DIR + queryId.toString() + "/output"+ "/" + 
sid + "/" +ta + "/output/" + partId;
    +    String params = String.format("qid=%s&sid=%s&p=%s&type=%s&ta=%s", 
queryId, sid, partId, "h", ta);
    +
    +    FSDataOutputStream stream =  LocalFileSystem.get(conf).create(new 
Path(dataPath), true);
         for (int i = 0; i < 100; i++) {
    -      data = ""+rnd.nextInt();
    -      writer.write(data);
    +      String data = ""+rnd.nextInt();
    +      stream.write(data.getBytes());
         }
    -    writer.flush();
    -    writer.close();
    +    stream.flush();
    +    stream.close();
    +
    +    URI uri = URI.create("http://127.0.0.1:"; + pullServerService.getPort() 
+ "/?" + params);
    +    final Fetcher fetcher = new Fetcher(conf, uri, new File(OUTPUT_DIR + 
"data"), channelFactory);
    +    assertEquals(TajoProtos.FetcherState.FETCH_INIT, fetcher.getState());
    +
    +    fetcher.get();
    +    assertEquals(TajoProtos.FetcherState.FETCH_FINISHED, 
fetcher.getState());
    +  }
     
    -    DataRetriever ret = new DirectoryRetriever(INPUT_DIR);
    -    final HttpDataServer server = new HttpDataServer(
    -        NetUtils.createSocketAddr("127.0.0.1:0"), ret);
    -    server.start();
    -    InetSocketAddress addr = server.getBindAddress();
    +  @Test
    +  public void testEmptyFileTask() throws Exception {
    +
    +    QueryId queryId = QueryIdFactory.NULL_QUERY_ID;
    +    String sid = "1";
    +    String ta = "1_0";
    +    String partId = "1";
    +
    +    String dataPath = INPUT_DIR + queryId.toString() + "/output"+ "/" + 
sid + "/" +ta + "/output/" + partId;
    +    String params = String.format("qid=%s&sid=%s&p=%s&type=%s&ta=%s", 
queryId, sid, partId, "h", ta);
    +
    +    Path inputPath = new Path(dataPath);
    +    if(LocalFileSystem.get(conf).exists(inputPath)){
    +      LocalFileSystem.get(conf).delete(new Path(dataPath), true);
    +    }
     
    -    URI uri = URI.create("http://127.0.0.1:"+addr.getPort() + "/data");
    -    ClientSocketChannelFactory channelFactory = 
RpcChannelFactory.createClientChannelFactory("Fetcher", 1);
    +    FSDataOutputStream stream =  LocalFileSystem.get(conf).create(new 
Path(dataPath).getParent(), true);
    +    stream.close();
     
    -    final Fetcher fetcher = new Fetcher(uri, new File(OUTPUT_DIR + 
"data"), channelFactory);
    +    URI uri = URI.create("http://127.0.0.1:"; + pullServerService.getPort() 
+ "/?" + params);
    +    final Fetcher fetcher = new Fetcher(conf, uri, new File(OUTPUT_DIR + 
"data"), channelFactory);
         assertEquals(TajoProtos.FetcherState.FETCH_INIT, fetcher.getState());
     
         fetcher.get();
         assertEquals(TajoProtos.FetcherState.FETCH_FINISHED, 
fetcher.getState());
    -    server.stop();
    +  }
    +
    +  @Test
    +  public void testFailureStatus() throws Exception {
    +    Random rnd = new Random();
    +
    +    QueryId queryId = QueryIdFactory.NULL_QUERY_ID;
    +    String sid = "1";
    +    String ta = "1_0";
    +    String partId = "1";
    +
    +    String dataPath = INPUT_DIR + queryId.toString() + "/output"+ "/" + 
sid + "/" +ta + "/output/" + partId;
    +    String params = String.format("qid=%s&sid=%s&p=%s&type=%s&ta=%s", 
queryId, sid, partId, "x", ta);
    --- End diff --
    
    The fetch failure will be caused by shuffle type character 'x' because 
'h','r', and 's' are only allowed. But, it is hard for usual contributors to 
know this fact. Could you add a brief comment about the fact? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to