You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by pr...@apache.org on 2010/01/04 20:37:08 UTC

svn commit: r895754 - in /hadoop/pig/branches/load-store-redesign: src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/ src/org/apache/pig/impl/builtin/ test/org/apache/pig/test/ test/org/apache/pig/test/data/GoldenFiles/

Author: pradeepkth
Date: Mon Jan  4 19:37:05 2010
New Revision: 895754

URL: http://svn.apache.org/viewvc?rev=895754&view=rev
Log:
PIG-1094: Fix unit tests corresponding to source changes so far - fixes for some order by,merge join and skew join related tests (thejas via pradeepkth)

Modified:
    hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/WeightedRangePartitioner.java
    hadoop/pig/branches/load-store-redesign/src/org/apache/pig/impl/builtin/GetMemNumRows.java
    hadoop/pig/branches/load-store-redesign/src/org/apache/pig/impl/builtin/PoissonSampleLoader.java
    hadoop/pig/branches/load-store-redesign/test/org/apache/pig/test/TestPoissonSampleLoader.java
    hadoop/pig/branches/load-store-redesign/test/org/apache/pig/test/data/GoldenFiles/MRC18.gld

Modified: hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/WeightedRangePartitioner.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/WeightedRangePartitioner.java?rev=895754&r1=895753&r2=895754&view=diff
==============================================================================
--- hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/WeightedRangePartitioner.java (original)
+++ hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/WeightedRangePartitioner.java Mon Jan  4 19:37:05 2010
@@ -29,6 +29,8 @@
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.Partitioner;
+import org.apache.pig.FuncSpec;
+import org.apache.pig.LoadFunc;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.HDataType;
 import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
@@ -37,6 +39,7 @@
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.InternalMap;
 import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.builtin.FindQuantiles;
 import org.apache.pig.impl.io.BufferedPositionedInputStream;
 import org.apache.pig.impl.io.FileLocalizer;
@@ -93,7 +96,6 @@
             throw new RuntimeException(this.getClass().getSimpleName() + " used but no quantiles found");
         
         try{
-            InputStream is = FileLocalizer.openDFSFile(quantilesFile,ConfigurationUtil.toProperties(job));
             ReadToEndLoader loader = new ReadToEndLoader(new BinStorage(), job, 
                     quantilesFile, 0);
             DataBag quantilesList;
@@ -114,17 +116,25 @@
                             new DiscreteProbabilitySampleGenerator(probVec));
                 }
             }
-            else
-            {
-                ArrayList<Pair<FileSpec, Boolean>> inp = (ArrayList<Pair<FileSpec, Boolean>>)ObjectSerializer.deserialize(job.get("pig.inputs", ""));
-                String inputFileName = inp.get(0).first.getFileName();
-                long inputSize = FileLocalizer.getSize(inputFileName);
-                if (inputSize!=0)
+            else {
+                ArrayList<FileSpec> inp = 
+                    (ArrayList<FileSpec>)
+                    ObjectSerializer.deserialize(job.get("pig.inputs", ""));
+                //order-by MR job will have only one input
+                FileSpec fileSpec = inp.get(0);
+                LoadFunc inpLoad =
+                    (LoadFunc)PigContext.instantiateFuncFromSpec(fileSpec.getFuncSpec());
+
+                ReadToEndLoader r2eLoad = new ReadToEndLoader(inpLoad, job, 
+                        fileSpec.getFileName(), 0);
+
+                if (r2eLoad.getNext() != null)
                 {
                     throw new RuntimeException("Empty samples file and non-empty input file");
                 }
                 // Otherwise, we do not put anything to weightedParts
             }
+            
         }catch (Exception e){
             throw new RuntimeException(e);
         }

Modified: hadoop/pig/branches/load-store-redesign/src/org/apache/pig/impl/builtin/GetMemNumRows.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/load-store-redesign/src/org/apache/pig/impl/builtin/GetMemNumRows.java?rev=895754&r1=895753&r2=895754&view=diff
==============================================================================
--- hadoop/pig/branches/load-store-redesign/src/org/apache/pig/impl/builtin/GetMemNumRows.java (original)
+++ hadoop/pig/branches/load-store-redesign/src/org/apache/pig/impl/builtin/GetMemNumRows.java Mon Jan  4 19:37:05 2010
@@ -54,8 +54,8 @@
     	
     	//  if this is specially marked tuple, get the number of rows
         int tSize = in.size();
-    	if(tSize >=2 &&
-    	        in.get(tSize-2).equals(PoissonSampleLoader.NUMROWS_TUPLE_MARKER)){
+    	if(tSize >=2 && 
+    	    PoissonSampleLoader.NUMROWS_TUPLE_MARKER.equals(in.get(tSize-2)) ){
     	    numRows = (Long)in.get(tSize-1);
     	}
     	

Modified: hadoop/pig/branches/load-store-redesign/src/org/apache/pig/impl/builtin/PoissonSampleLoader.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/load-store-redesign/src/org/apache/pig/impl/builtin/PoissonSampleLoader.java?rev=895754&r1=895753&r2=895754&view=diff
==============================================================================
--- hadoop/pig/branches/load-store-redesign/src/org/apache/pig/impl/builtin/PoissonSampleLoader.java (original)
+++ hadoop/pig/branches/load-store-redesign/src/org/apache/pig/impl/builtin/PoissonSampleLoader.java Mon Jan  4 19:37:05 2010
@@ -34,8 +34,10 @@
  */
 public class PoissonSampleLoader extends SampleLoader {
     
-    // marker string for special row with total number or rows. 
-    // this will be value of first column in the special row
+    // marker string to mark the last sample row, which has total number or rows
+    // seen by this map instance
+    // this string will be in the 2nd last column of the last sample row
+    // it is used by GetMemNumRows
     public static final String NUMROWS_TUPLE_MARKER = 
         "\u4956\u3838_pig_inTeRnal-spEcial_roW_num_tuple3kt579CFLehkblah";
     
@@ -90,7 +92,7 @@
             // were read, nothing more to read 
             return null;
         }
-        
+
 
         if(skipInterval == -1){
             //select first tuple as sample and calculate
@@ -101,9 +103,9 @@
             long availRedMem = (long) (Runtime.getRuntime().maxMemory() * heapPerc);
             memToSkipPerSample = availRedMem/sampleRate;
             updateSkipInterval(t);
-                
+
             rowNum++;
-                newSample = t;
+            newSample = t;
         }
 
         // skip tuples
@@ -113,16 +115,16 @@
             }
             rowNum++;
         }
-        
+
         // skipped enough, get new sample
         Tuple t = loader.getNext();
         if(t == null)
             return createNumRowTuple(newSample);
         updateSkipInterval(t);
         rowNum++;
-            Tuple currentSample = newSample;
-            newSample = t;
-            return currentSample;
+        Tuple currentSample = newSample;
+        newSample = t;
+        return currentSample;
     }
 
     /**
@@ -141,7 +143,7 @@
             // very few samples being sampled. Sampling a little extra is OK
         if(numRowsSampled < 5)
             skipInterval = skipInterval/(10-numRowsSampled);
-            ++numRowsSampled;
+        ++numRowsSampled;
 
     }
 

Modified: hadoop/pig/branches/load-store-redesign/test/org/apache/pig/test/TestPoissonSampleLoader.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/load-store-redesign/test/org/apache/pig/test/TestPoissonSampleLoader.java?rev=895754&r1=895753&r2=895754&view=diff
==============================================================================
--- hadoop/pig/branches/load-store-redesign/test/org/apache/pig/test/TestPoissonSampleLoader.java (original)
+++ hadoop/pig/branches/load-store-redesign/test/org/apache/pig/test/TestPoissonSampleLoader.java Mon Jan  4 19:37:05 2010
@@ -32,10 +32,10 @@
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.builtin.PoissonSampleLoader;
 import org.apache.pig.impl.util.Pair;
+import org.apache.pig.impl.io.FileSpec;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
-import org.apache.pig.impl.io.FileSpec;
 
 
 public class TestPoissonSampleLoader extends TestCase{
@@ -86,33 +86,19 @@
     }
 
     @Test
-    public void testComputeSamples() throws IOException{
-        FileSpec fs = new FileSpec(INPUT_FILE1, new FuncSpec(PigStorage.class.getName()));
-
-        ArrayList<Pair<FileSpec, Boolean>> inputs = new ArrayList<Pair<FileSpec, Boolean> >();
-        inputs.add(new Pair<FileSpec, Boolean>(fs, true));
-
-        // Use 100 as a default value;
-        PoissonSampleLoader ps = new PoissonSampleLoader((new FuncSpec(PigStorage.class.getName())).toString(), "100");
-
-        // Get the number of samples for the file
-        ps.computeSamples(inputs, pigServer.getPigContext());
-
-        if (ps.getNumSamples() != 3) {
-            fail("Compute samples returned the wrong number of samples");
+    public void testNumSamples() throws IOException {
+        pigServer.registerQuery("A = Load '"+INPUT_FILE1+"' Using PoissonSampleLoader('PigStorage()', '100');");
+        Iterator<Tuple> iter = pigServer.openIterator("A");
+        int count = 0;
+        while(iter.hasNext()){
+            count++;
+            iter.next();
         }
+        assertEquals(count, 1);
     }
 
     /*
-     * FIXME This currently tests for 5 elements because PoissonSampleLoader
-     * only produces a single sample for the test data, and the last sample has
-     * extra information appended in PoissonSampleLoader. 
-     * 
-     * This is incorrect. The proper number of samples should be > 1, and therefore
-     * the first sample should only have 3 elements.
-     * 
-     * See PIG-1062 and PIG-1149 for more information.
-     * 
+     * Test use of LoadFunc with parameters as argument to PoissonSampleLoader
      */
     @Test
     public void testInstantiation() throws IOException {
@@ -122,6 +108,4 @@
         assertEquals(5, iter.next().size());
     }
 
-
-
 }
\ No newline at end of file

Modified: hadoop/pig/branches/load-store-redesign/test/org/apache/pig/test/data/GoldenFiles/MRC18.gld
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/load-store-redesign/test/org/apache/pig/test/data/GoldenFiles/MRC18.gld?rev=895754&r1=895753&r2=895754&view=diff
==============================================================================
--- hadoop/pig/branches/load-store-redesign/test/org/apache/pig/test/data/GoldenFiles/MRC18.gld (original)
+++ hadoop/pig/branches/load-store-redesign/test/org/apache/pig/test/data/GoldenFiles/MRC18.gld Mon Jan  4 19:37:05 2010
@@ -1,10 +1,10 @@
 MapReduce(-1,PigStorage) - scope-125:
 Reduce Plan Empty
-|   Store(file:/tmp:org.apache.pig.builtin.PigStorage) - scope-124
+|   Store(file:///tmp:org.apache.pig.builtin.PigStorage) - scope-124
 |   |
 |   |---MergeJoin[tuple] - scope-121
 |       |
-|       |---Load(file:/tmp/input1:org.apache.pig.builtin.PigStorage) - scope-117
+|       |---Load(file:///tmp/input1:org.apache.pig.builtin.PigStorage) - scope-117
 |
 |---MapReduce(1,PigStorage) - scope-126:
     |   Store(file:/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - scope-133
@@ -20,4 +20,4 @@
     |   |   |
     |   |   Constant(all) - scope-127
     |   |
-    |   |---Load(file:/tmp/input2:org.apache.pig.impl.builtin.MergeJoinIndexer('org.apache.pig.builtin.PigStorage','kmonaaafhdhcaabdgkgbhggbcohfhegjgmcoebhchcgbhjemgjhdhehiibncbnjjmhgbjnadaaabejaaaehdgjhkgfhihaaaaaaaabhhaeaaaaaaabhdhcaaeogphcghcogbhagbgdgigfcohagjghcogcgbgdglgfgogecogigbgegpgphacogfhigfgdhfhegjgpgogfgoghgjgogfcohagihjhdgjgdgbgmemgbhjgfhccohagmgbgohdcofagihjhdgjgdgbgmfagmgbgoaaaaaaaaaaaaaaabacaaabfkaaangfgogeepggebgmgmejgohahfhehihcaacfgphcghcogbhagbgdgigfcohagjghcogjgnhagmcohagmgbgocoephagfhcgbhegphcfagmgbgohlinhnlnngaiiihoacaaagemaaakgneghcgpgnefgeghgfhdheaacdemgphcghcpgbhagbgdgigfcphagjghcpgjgnhagmcphfhegjgmcpenhfgmhegjengbhadlemaaafgnelgfhjhdheaaapemgkgbhggbcphfhegjgmcpengbhadlemaaahgnemgfgbhggfhdheaabaemgkgbhggbcphfhegjgmcpemgjhdhedlemaaaegnephahdhbaahoaaafemaaaggnfcgpgphehdhbaahoaaagemaaaignfegpefgeghgfhdhbaahoaaaehihahdhcaacbgphcghcogbhagbgdgigfcohagjghcogjgnhagmcohfhegjgmcoenhfgmhegjengbhaaaaaaaaaaaaaaaacacaaabemaaaegnengbhahbaahoaaafhihahdhcaabbgkgbh
 ggbcohfhegjgmcoeigbhdgiengbhaafahnkmbmdbgganbadaaacegaaakgmgpgbgeeggbgdhegphcejaaajhegihcgfhdgigpgmgehihadpeaaaaaaaaaaaamhhaiaaaaaabaaaaaaaaahihdhbaahoaaakdpeaaaaaaaaaaaamhhaiaaaaaabaaaaaaaabhdhcaacegphcghcogbhagbgdgigfcohagjghcogjgnhagmcohagmgbgocoephagfhcgbhegphcelgfhjaaaaaaaaaaaaaaabacaaacekaaacgjgeemaaafhdgdgphagfheaabcemgkgbhggbcpgmgbgoghcpfdhehcgjgoghdlhihaaaaaaaaaaaaaaahiheaaafhdgdgphagfhdhcaafjgphcghcogbhagbgdgigfcohagjghcogcgbgdglgfgogecogigbgegpgphacogfhigfgdhfhegjgpgogfgoghgjgogfcohagihjhdgjgdgbgmemgbhjgfhccogfhihahcgfhdhdgjgpgoephagfhcgbhegphchdcofaepfahcgpgkgfgdheaaaaaaaaaaaaaaabacaaaffkaaakgphggfhcgmgpgbgegfgefkaabfhahcgpgdgfhdhdgjgoghecgbghepggfehfhagmgfhdfkaabehcgfhdhfgmhefdgjgoghgmgffehfhagmgfecgbghfkaaaehdhegbhcemaaahgdgpgmhfgngohdheaabfemgkgbhggbcphfhegjgmcpebhchcgbhjemgjhdhedlhihcaagcgphcghcogbhagbgdgigfcohagjghcogcgbgdglgfgogecogigbgegpgphacogfhigfgdhfhegjgpgogfgoghgjgogfcohagihjhdgjgdgbgmemgbhjgfhccogfhihahcgfhdhdgjgpgoephagfhcgbhegphchdcoefhihahcgfhdhd
 gjgpgoephagfhcgbhegphcaaaaaaaaaaaaaaabacaaabemaaadgmgpghheaacaemgphcghcpgbhagbgdgigfcpgdgpgngngpgohdcpgmgpghghgjgoghcpemgpghdlhihcaaemgphcghcogbhagbgdgigfcohagjghcogcgbgdglgfgogecogigbgegpgphacogfhigfgdhfhegjgpgogfgoghgjgogfcohagihjhdgjgdgbgmemgbhjgfhccofagihjhdgjgdgbgmephagfhcgbhegphcaaaaaaaaaaaaaaabacaaamfkaaafgbgdgdhfgnfkaaangjgohahfheebhehegbgdgigfgeejaabehcgfhbhfgfhdhegfgefagbhcgbgmgmgfgmgjhdgnecaaakhcgfhdhfgmhefehjhagfemaaafgbgmgjgbhdhbaahoaaaoemaaafgjgohahfheheaablemgphcghcpgbhagbgdgigfcphagjghcpgegbhegbcpfehfhagmgfdlemaaaggjgohahfhehdhbaahoaaagemaaangmgjgogfgbghgffehcgbgdgfhcheaachemgphcghcpgbhagbgdgigfcphagjghcphagfgocphfhegjgmcpemgjgogfgbghgffehcgbgdgfhcdlemaaadgmgpghhbaahoaabeemaaahgphfhehahfhehdhbaahoaaagemaaakhagbhcgfgohefagmgbgoheaafaemgphcghcpgbhagbgdgigfcphagjghcpgcgbgdglgfgogecpgigbgegpgphacpgfhigfgdhfhegjgpgogfgoghgjgogfcphagihjhdgjgdgbgmemgbhjgfhccphagmgbgohdcpfagihjhdgjgdgbgmfagmgbgodlemaaadhcgfhdheaaeeemgphcghcpgbhagbgdgigfcphagjghcpgcgbgdglgfgogecpgigbg
 egpgphacpgfhigfgdhfhegjgpgogfgoghgjgogfcphagihjhdgjgdgbgmemgbhjgfhccpfcgfhdhfgmhedlhihcaacbgphcghcogbhagbgdgigfcohagjghcogjgnhagmcohagmgbgocoephagfhcgbhegphcaaaaaaaaaaaaaaabacaaabemaaaegnelgfhjheaacgemgphcghcpgbhagbgdgigfcphagjghcpgjgnhagmcphagmgbgocpephagfhcgbhegphcelgfhjdlhihahbaahoaaapaaaappppppppdchahahahahdhcaaclgphcghcogbhagbgdgigfcogdgpgngngpgohdcogmgpghghgjgoghcogjgnhagmcoemgpghdeekemgpghghgfhccikmpnoicknfncdiacaaabemaaaegogbgngfhbaahoaaaohihaheaafjgphcghcogbhagbgdgigfcohagjghcogcgbgdglgfgogecogigbgegpgphacogfhigfgdhfhegjgpgogfgoghgjgogfcohagihjhdgjgdgbgmemgbhjgfhccogfhihahcgfhdhdgjgpgoephagfhcgbhegphchdcofaepfahcgpgkgfgdhehahahdhcaaecgphcghcogbhagbgdgigfcohagjghcogcgbgdglgfgogecogigbgegpgphacogfhigfgdhfhegjgpgogfgoghgjgogfcohagihjhdgjgdgbgmemgbhjgfhccofcgfhdhfgmheaaaaaaaaaaaaaaabacaaacecaaamhcgfhehfhcgofdhegbhehfhdemaaaghcgfhdhfgmheheaabcemgkgbhggbcpgmgbgoghcpepgcgkgfgdhedlhihaachahbaahoaaboaaaaaaaahdhbaahoaaaaaaaaaaabhhaeaaaaaaabhdhcaabbgkgbhggbcogmgbgoghcoejgohegf
 ghgfhcbcockakephibihdiacaaabejaaafhggbgmhfgfhihcaabagkgbhggbcogmgbgoghcoeohfgngcgfhcigkmjfbnaljeoailacaaaahihaaaaaaaaahihihdhbaahoaaaaaaaaaaabhhaeaaaaaaakhbaahoaabmhihdhbaahoaaakdpeaaaaaaaaaaaamhhaiaaaaaabaaaaaaaabhbaahoaabmhbaahoaaaphihdhbaahoaaaaaaaaaaaahhaeaaaaaaakhihdhbaahoaaaihdhbaahoaaakdpeaaaaaaaaaaaamhhaiaaaaaabaaaaaaaaahiaahi','')) - scope-118
\ No newline at end of file
+    |   |---Load(file:///tmp/input2:org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MergeJoinIndexer('org.apache.pig.builtin.PigStorage','kmonaaafhdhcaabdgkgbhggbcohfhegjgmcoebhchcgbhjemgjhdhehiibncbnjjmhgbjnadaaabejaaaehdgjhkgfhihaaaaaaaabhhaeaaaaaaabhdhcaaeogphcghcogbhagbgdgigfcohagjghcogcgbgdglgfgogecogigbgegpgphacogfhigfgdhfhegjgpgogfgoghgjgogfcohagihjhdgjgdgbgmemgbhjgfhccohagmgbgohdcofagihjhdgjgdgbgmfagmgbgoaaaaaaaaaaaaaaabacaaabfkaaangfgogeepggebgmgmejgohahfhehihcaacfgphcghcogbhagbgdgigfcohagjghcogjgnhagmcohagmgbgocoephagfhcgbhegphcfagmgbgohlinhnlnngaiiihoacaaagemaaakgneghcgpgnefgeghgfhdheaacdemgphcghcpgbhagbgdgigfcphagjghcpgjgnhagmcphfhegjgmcpenhfgmhegjengbhadlemaaafgnelgfhjhdheaaapemgkgbhggbcphfhegjgmcpengbhadlemaaahgnemgfgbhggfhdheaabaemgkgbhggbcphfhegjgmcpemgjhdhedlemaaaegnephahdhbaahoaaafemaaaggnfcgpgphehdhbaahoaaagemaaaignfegpefgeghgfhdhbaahoaaaehihahdhcaacbgphcghcogbhagbgdgigfcohagjghcogjgnhagmcohfhegjgmcoenhfgmhegjengbhaaaaaaaaaaaaaaaacacaaabemaaae
 gnengbhahbaahoaaafhihahdhcaabbgkgbhggbcohfhegjgmcoeigbhdgiengbhaafahnkmbmdbgganbadaaacegaaakgmgpgbgeeggbgdhegphcejaaajhegihcgfhdgigpgmgehihadpeaaaaaaaaaaaamhhaiaaaaaabaaaaaaaaahihdhbaahoaaakdpeaaaaaaaaaaaamhhaiaaaaaabaaaaaaaabhdhcaacegphcghcogbhagbgdgigfcohagjghcogjgnhagmcohagmgbgocoephagfhcgbhegphcelgfhjaaaaaaaaaaaaaaabacaaacekaaacgjgeemaaafhdgdgphagfheaabcemgkgbhggbcpgmgbgoghcpfdhehcgjgoghdlhihaaaaaaaaaaaaaaahiheaaafhdgdgphagfhdhcaafjgphcghcogbhagbgdgigfcohagjghcogcgbgdglgfgogecogigbgegpgphacogfhigfgdhfhegjgpgogfgoghgjgogfcohagihjhdgjgdgbgmemgbhjgfhccogfhihahcgfhdhdgjgpgoephagfhcgbhegphchdcofaepfahcgpgkgfgdheaaaaaaaaaaaaaaabacaaaffkaaakgphggfhcgmgpgbgegfgefkaabfhahcgpgdgfhdhdgjgoghecgbghepggfehfhagmgfhdfkaabehcgfhdhfgmhefdgjgoghgmgffehfhagmgfecgbghfkaaaehdhegbhcemaaahgdgpgmhfgngohdheaabfemgkgbhggbcphfhegjgmcpebhchcgbhjemgjhdhedlhihcaagcgphcghcogbhagbgdgigfcohagjghcogcgbgdglgfgogecogigbgegpgphacogfhigfgdhfhegjgpgogfgoghgjgogfcohagihjhdgjgdgbgmemgbhjgfhccogfhihahcgfhdhdgjgpg
 oephagfhcgbhegphchdcoefhihahcgfhdhdgjgpgoephagfhcgbhegphcaaaaaaaaaaaaaaabacaaabemaaadgmgpghheaacaemgphcghcpgbhagbgdgigfcpgdgpgngngpgohdcpgmgpghghgjgoghcpemgpghdlhihcaaemgphcghcogbhagbgdgigfcohagjghcogcgbgdglgfgogecogigbgegpgphacogfhigfgdhfhegjgpgogfgoghgjgogfcohagihjhdgjgdgbgmemgbhjgfhccofagihjhdgjgdgbgmephagfhcgbhegphcaaaaaaaaaaaaaaabacaaamfkaaafgbgdgdhfgnfkaaangjgohahfheebhehegbgdgigfgeejaabehcgfhbhfgfhdhegfgefagbhcgbgmgmgfgmgjhdgnecaaakhcgfhdhfgmhefehjhagfemaaafgbgmgjgbhdhbaahoaaaoemaaafgjgohahfheheaablemgphcghcpgbhagbgdgigfcphagjghcpgegbhegbcpfehfhagmgfdlemaaaggjgohahfhehdhbaahoaaagemaaangmgjgogfgbghgffehcgbgdgfhcheaachemgphcghcpgbhagbgdgigfcphagjghcphagfgocphfhegjgmcpemgjgogfgbghgffehcgbgdgfhcdlemaaadgmgpghhbaahoaabeemaaahgphfhehahfhehdhbaahoaaagemaaakhagbhcgfgohefagmgbgoheaafaemgphcghcpgbhagbgdgigfcphagjghcpgcgbgdglgfgogecpgigbgegpgphacpgfhigfgdhfhegjgpgogfgoghgjgogfcphagihjhdgjgdgbgmemgbhjgfhccphagmgbgohdcpfagihjhdgjgdgbgmfagmgbgodlemaaadhcgfhdheaaeeemgphcghcpgbhagbgd
 gigfcphagjghcpgcgbgdglgfgogecpgigbgegpgphacpgfhigfgdhfhegjgpgogfgoghgjgogfcphagihjhdgjgdgbgmemgbhjgfhccpfcgfhdhfgmhedlhihcaacbgphcghcogbhagbgdgigfcohagjghcogjgnhagmcohagmgbgocoephagfhcgbhegphcaaaaaaaaaaaaaaabacaaabemaaaegnelgfhjheaacgemgphcghcpgbhagbgdgigfcphagjghcpgjgnhagmcphagmgbgocpephagfhcgbhegphcelgfhjdlhihahbaahoaaapaaaappppppppdchahahahahdhcaaclgphcghcogbhagbgdgigfcogdgpgngngpgohdcogmgpghghgjgoghcogjgnhagmcoemgpghdeekemgpghghgfhccikmpnoicknfncdiacaaabemaaaegogbgngfhbaahoaaaohihaheaafjgphcghcogbhagbgdgigfcohagjghcogcgbgdglgfgogecogigbgegpgphacogfhigfgdhfhegjgpgogfgoghgjgogfcohagihjhdgjgdgbgmemgbhjgfhccogfhihahcgfhdhdgjgpgoephagfhcgbhegphchdcofaepfahcgpgkgfgdhehahahdhcaaecgphcghcogbhagbgdgigfcohagjghcogcgbgdglgfgogecogigbgegpgphacogfhigfgdhfhegjgpgogfgoghgjgogfcohagihjhdgjgdgbgmemgbhjgfhccofcgfhdhfgmheaaaaaaaaaaaaaaabacaaacecaaamhcgfhehfhcgofdhegbhehfhdemaaaghcgfhdhfgmheheaabcemgkgbhggbcpgmgbgoghcpepgcgkgfgdhedlhihaachahbaahoaaboaaaaaaaahdhbaahoaaaaaaaaaaabhhaeaaaaaaabh
 dhcaabbgkgbhggbcogmgbgoghcoejgohegfghgfhcbcockakephibihdiacaaabejaaafhggbgmhfgfhihcaabagkgbhggbcogmgbgoghcoeohfgngcgfhcigkmjfbnaljeoailacaaaahihaaaaaaaaahihihdhbaahoaaaaaaaaaaabhhaeaaaaaaakhbaahoaabmhihdhbaahoaaakdpeaaaaaaaaaaaamhhaiaaaaaabaaaaaaaabhbaahoaabmhbaahoaaaphihdhbaahoaaaaaaaaaaaahhaeaaaaaaakhihdhbaahoaaaihdhbaahoaaakdpeaaaaaaaaaaaamhhaiaaaaaabaaaaaaaaahiaahi','')) - scope-118
\ No newline at end of file