You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oodt.apache.org by bf...@apache.org on 2012/03/12 23:57:23 UTC

svn commit: r1299921 - in /oodt/trunk/pge: ./ src/main/java/org/apache/oodt/cas/pge/ src/main/java/org/apache/oodt/cas/pge/config/ src/main/java/org/apache/oodt/cas/pge/metadata/ src/main/java/org/apache/oodt/cas/pge/writers/ src/test/org/apache/oodt/c...

Author: bfoster
Date: Mon Mar 12 22:57:22 2012
New Revision: 1299921

URL: http://svn.apache.org/viewvc?rev=1299921&view=rev
Log:
- Port wengine ControlMetadata to cas-pge's PgeMetadata

-------------
OODT-403

Added:
    oodt/trunk/pge/src/test/org/apache/oodt/cas/pge/metadata/
    oodt/trunk/pge/src/test/org/apache/oodt/cas/pge/metadata/TestPgeMetadata.java   (with props)
Modified:
    oodt/trunk/pge/pom.xml
    oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/PGETaskInstance.java
    oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/config/XmlFilePgeConfigBuilder.java
    oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/metadata/PgeMetadata.java
    oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/writers/PcsMetFileWriter.java

Modified: oodt/trunk/pge/pom.xml
URL: http://svn.apache.org/viewvc/oodt/trunk/pge/pom.xml?rev=1299921&r1=1299920&r2=1299921&view=diff
==============================================================================
--- oodt/trunk/pge/pom.xml (original)
+++ oodt/trunk/pge/pom.xml Mon Mar 12 22:57:22 2012
@@ -111,6 +111,11 @@ the License.
       <version>${project.parent.version}</version>
     </dependency>
     <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <version>10.0.1</version>
+    </dependency>
+    <dependency>
       <groupId>commons-lang</groupId>
       <artifactId>commons-lang</artifactId>
       <version>2.1</version>

Modified: oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/PGETaskInstance.java
URL: http://svn.apache.org/viewvc/oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/PGETaskInstance.java?rev=1299921&r1=1299920&r2=1299921&view=diff
==============================================================================
--- oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/PGETaskInstance.java (original)
+++ oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/PGETaskInstance.java Mon Mar 12 22:57:22 2012
@@ -92,14 +92,14 @@ public class PGETaskInstance implements 
             WorkflowTaskConfiguration config) throws InstantiationException {
         try {
             // merge metadata
-            this.pgeMetadata = new PgeMetadata(metadata, config);
+            this.pgeMetadata = createPgeMetadata(metadata, config);
             
             // create PgeConfig
             this.pgeConfig = this.createPgeConfig();
 
             // load property adder
             String propertyAdderClasspath = this.pgeMetadata
-                    .getMetadataValue(PgeTaskMetadataKeys.PROPERTY_ADDER_CLASSPATH);
+                    .getMetadata(PgeTaskMetadataKeys.PROPERTY_ADDER_CLASSPATH);
             if (propertyAdderClasspath != null
                     && !propertyAdderClasspath.equals(""))
                 this.runPropertyAdder(this
@@ -107,9 +107,9 @@ public class PGETaskInstance implements 
 
             // configure workflow manager
             wm = new XmlRpcWorkflowManagerClient(new URL(this.pgeMetadata
-                    .getMetadataValue(PcsMetadataKeys.WORKFLOW_MANAGER_URL)));
+                    .getMetadata(PcsMetadataKeys.WORKFLOW_MANAGER_URL)));
             workflowInstId = this.pgeMetadata
-                    .getMetadataValue(CoreMetKeys.WORKFLOW_INST_ID);
+                    .getMetadata(CoreMetKeys.WORKFLOW_INST_ID);
 
         } catch (Exception e) {
             e.printStackTrace();
@@ -118,6 +118,16 @@ public class PGETaskInstance implements 
         }
     }
 
+    protected PgeMetadata createPgeMetadata(Metadata dynMetadata,
+          WorkflowTaskConfiguration config) {
+       Metadata staticMetadata = new Metadata();
+       for (Object key : config.getProperties().keySet()) {
+          staticMetadata.addMetadata((String) key,
+                config.getProperty((String) key));
+       }
+       return new PgeMetadata(staticMetadata, dynMetadata);
+    }
+
     protected ConfigFilePropertyAdder loadPropertyAdder(
             String propertyAdderClasspath) throws Exception {
         return (ConfigFilePropertyAdder) Class.forName(propertyAdderClasspath)
@@ -190,7 +200,7 @@ public class PGETaskInstance implements 
         SciPgeConfigFileWriter writer = (SciPgeConfigFileWriter) Class.forName(
                 dynamicConfigFile.getWriterClass()).newInstance();
         writer.createConfigFile(dynamicConfigFile.getFilePath(),
-                this.pgeMetadata.getMetadata(), dynamicConfigFile.getArgs());
+                this.pgeMetadata.asMetadata(), dynamicConfigFile.getArgs());
     }
 
     protected void processOutput() throws FileNotFoundException, IOException {
@@ -220,13 +230,13 @@ public class PGETaskInstance implements 
                 }
                 if (outputMetadata.getAllKeys().size() > 0)
                 	this.writeFromMetadata(outputMetadata, createdFile.getAbsolutePath() 
-                			+ "." + this.pgeMetadata.getMetadataValue(PcsMetadataKeys.MET_FILE_EXT));
+                			+ "." + this.pgeMetadata.getMetadata(PcsMetadataKeys.MET_FILE_EXT));
             }
         }
     }
 
     protected File renameFile(File file, RenamingConv renamingConv) throws Exception {
-    	Metadata curMetadata = this.pgeMetadata.getMetadata();
+    	Metadata curMetadata = this.pgeMetadata.asMetadata();
     	curMetadata.replaceMetadata(renamingConv.getTmpReplaceMet());
     	String newFileName = PathUtils.doDynamicReplacement(renamingConv.getRenamingString(), curMetadata);
     	File newFile = new File(file.getParentFile(), newFileName);
@@ -262,14 +272,14 @@ public class PGETaskInstance implements 
 
     protected String getPgeScriptName() {
         return "sciPgeExeScript_"
-                + this.pgeMetadata.getMetadataValue(PgeTaskMetadataKeys.NAME);
+                + this.pgeMetadata.getMetadata(PgeTaskMetadataKeys.NAME);
     }
 
     protected Handler initializePgeLogHandler() throws SecurityException,
 			IOException {
     	FileHandler handler = null;
 		String logFilePattern = this.pgeMetadata
-				.getMetadataValue(PgeTaskMetadataKeys.LOG_FILE_PATTERN);
+				.getMetadata(PgeTaskMetadataKeys.LOG_FILE_PATTERN);
 		if (logFilePattern != null) {
 			LOG.log(Level.INFO, "Creating Log Handler to capture pge output to file '"
 							+ logFilePattern + "'");
@@ -284,7 +294,7 @@ public class PGETaskInstance implements 
     protected Logger initializePgeLogger(Handler handler) {
     	if (handler != null) {
 	    	Logger pgeLogger = Logger.getLogger(this.pgeMetadata
-					.getMetadataValue(PgeTaskMetadataKeys.NAME)
+					.getMetadata(PgeTaskMetadataKeys.NAME)
 					+ System.currentTimeMillis());
 			pgeLogger.addHandler(handler);
 			return pgeLogger;
@@ -326,7 +336,7 @@ public class PGETaskInstance implements 
            
             
             long endTime = System.currentTimeMillis();
-            this.pgeMetadata.addCustomMetadata(PgeTaskMetadataKeys.PGE_RUNTIME,
+            this.pgeMetadata.replaceMetadata(PgeTaskMetadataKeys.PGE_RUNTIME,
                     (endTime - startTime) + "");
 
         } catch (Exception e) {
@@ -359,35 +369,35 @@ public class PGETaskInstance implements 
     protected void setCrawlerConfigurations(StdProductCrawler crawler)
             throws Exception {
         crawler.setMetFileExtension(this.pgeMetadata
-                .getMetadataValue(PcsMetadataKeys.MET_FILE_EXT));
+                .getMetadata(PcsMetadataKeys.MET_FILE_EXT));
         crawler
                 .setClientTransferer(this.pgeMetadata
-                        .getMetadataValue(PcsMetadataKeys.CLIENT_TRANSFER_SERVICE_FACTORY));
+                        .getMetadata(PcsMetadataKeys.CLIENT_TRANSFER_SERVICE_FACTORY));
         crawler.setFilemgrUrl(this.pgeMetadata
-                .getMetadataValue(PcsMetadataKeys.FILE_MANAGER_URL));
+                .getMetadata(PcsMetadataKeys.FILE_MANAGER_URL));
         String actionRepoFile = this.pgeMetadata
-                .getMetadataValue(PcsMetadataKeys.ACTION_REPO_FILE);
+                .getMetadata(PcsMetadataKeys.ACTION_REPO_FILE);
         if (actionRepoFile != null && !actionRepoFile.equals("")) {
             crawler.setApplicationContext(new FileSystemXmlApplicationContext(
                     actionRepoFile));
             crawler.setActionIds(this.pgeMetadata
-                    .getMetadataValues(PcsMetadataKeys.ACTION_IDS));
+                    .getAllMetadata(PcsMetadataKeys.ACTION_IDS));
         }
         crawler.setRequiredMetadata(this.pgeMetadata
-                .getMetadataValues(PcsMetadataKeys.REQUIRED_METADATA));
+                .getAllMetadata(PcsMetadataKeys.REQUIRED_METADATA));
         String crawlForDirsString = this.pgeMetadata
-                .getMetadataValue(PcsMetadataKeys.CRAWLER_CRAWL_FOR_DIRS);
+                .getMetadata(PcsMetadataKeys.CRAWLER_CRAWL_FOR_DIRS);
         boolean crawlForDirs = (crawlForDirsString != null) ? crawlForDirsString
                 .toLowerCase().equals("true")
                 : false;
         String recurString = this.pgeMetadata
-                .getMetadataValue(PcsMetadataKeys.CRAWLER_RECUR);
+                .getMetadata(PcsMetadataKeys.CRAWLER_RECUR);
         boolean recur = (recurString != null) ? recurString.toLowerCase()
                 .equals("true") : true;
         crawler.setCrawlForDirs(crawlForDirs);
         crawler.setNoRecur(!recur);
     	LOG.log(Level.INFO, "Passing Workflow Metadata to CAS-Crawler as global metadata . . .");
-    	crawler.setGlobalMetadata(this.pgeMetadata.getMetadata(PgeMetadata.DYN));
+    	crawler.setGlobalMetadata(this.pgeMetadata.asMetadata(PgeMetadata.Type.DYNAMIC));
     }
 
     protected void runIngestCrawler(StdProductCrawler crawler,
@@ -396,7 +406,7 @@ public class PGETaskInstance implements 
 		try {
 			this.updateStatus(PgeTaskMetadataKeys.CRAWLING);
 			boolean attemptIngestAll = Boolean.parseBoolean(this.pgeMetadata
-					.getMetadataValue(PgeTaskMetadataKeys.ATTEMPT_INGEST_ALL));
+					.getMetadata(PgeTaskMetadataKeys.ATTEMPT_INGEST_ALL));
 			for (File crawlDir : crawlDirs) {
 				currentDir = crawlDir;
 				LOG.log(Level.INFO,
@@ -440,7 +450,7 @@ public class PGETaskInstance implements 
     }
     
     protected void updateDynamicMetadata() {
-        this.pgeMetadata.commitWorkflowMetadataKeys();
+        this.pgeMetadata.commitMarkedDynamicMetadataKeys();
     }
 
     public void run(Metadata metadata, WorkflowTaskConfiguration config)
@@ -457,5 +467,4 @@ public class PGETaskInstance implements 
                     + e.getMessage(), e);
         }
     }
-
 }

Modified: oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/config/XmlFilePgeConfigBuilder.java
URL: http://svn.apache.org/viewvc/oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/config/XmlFilePgeConfigBuilder.java?rev=1299921&r1=1299920&r2=1299921&view=diff
==============================================================================
--- oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/config/XmlFilePgeConfigBuilder.java (original)
+++ oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/config/XmlFilePgeConfigBuilder.java Mon Mar 12 22:57:22 2012
@@ -14,10 +14,41 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.oodt.cas.pge.config;
 
+//OODT static imports
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.ARGS_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.CMD_TAG;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.CREATE_BEFORE_EXEC_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.CUSTOM_METADATA_TAG;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.DIR_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.DIR_TAG;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.DYN_INPUT_FILES_TAG;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.ENV_REPLACE_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.ENV_REPLACE_NO_RECUR_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.EXE_TAG;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.FILES_TAG;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.FILE_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.FILE_TAG;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.IMPORT_TAG;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.KEYREF_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.KEY_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.KEY_GEN_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.METADATA_TAG;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.MET_FILE_WRITER_CLASS_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.NAMESPACE_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.NAME_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.NAMING_EXPR_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.OUTPUT_TAG;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.PATH_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.REGEX_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.RENAMING_CONV_TAG;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.SHELL_TYPE_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.SPLIT_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.VAL_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.WORKFLOW_MET_ATTR;
+import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.WRITER_CLASS_ATTR;
+
 //JDK imports
 import java.io.File;
 import java.io.FileInputStream;
@@ -33,31 +64,26 @@ import java.util.logging.Level;
 import java.util.logging.Logger;
 
 //OODT imports
-import org.apache.oodt.cas.metadata.Metadata;
-import org.apache.oodt.cas.metadata.util.PathUtils;
-import org.apache.oodt.commons.xml.XMLUtils;
 import org.apache.oodt.cas.filemgr.system.XmlRpcFileManagerClient;
 import org.apache.oodt.cas.filemgr.util.QueryUtils;
 import org.apache.oodt.cas.filemgr.util.SqlParser;
+import org.apache.oodt.cas.metadata.Metadata;
+import org.apache.oodt.cas.metadata.util.PathUtils;
 import org.apache.oodt.cas.pge.metadata.PcsMetadataKeys;
 import org.apache.oodt.cas.pge.metadata.PgeMetadata;
 import org.apache.oodt.cas.pge.metadata.PgeTaskMetadataKeys;
-import static org.apache.oodt.cas.pge.config.PgeConfigMetKeys.*;
+import org.apache.oodt.commons.xml.XMLUtils;
 
-//DOM imports
+//W3C imports
 import org.w3c.dom.DOMException;
 import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
 
 /**
- * 
- * @author bfoster
- * @version $Revision$
- * 
- * <p>
  * An implementation of the {@link PgeConfigBuilder} that reads an XML file
- * representation of the {@link PgeConfig}
- * </p>.
+ * representation of the {@link PgeConfig}.
+ *
+ * @author bfoster (Brian Foster)
  */
 public class XmlFilePgeConfigBuilder implements PgeConfigBuilder {
     
@@ -67,15 +93,14 @@ public class XmlFilePgeConfigBuilder imp
         try {
             PgeConfig pgeConfig = new PgeConfig();
             this.buildImports(this.fillIn(pgeMetadata
-                    .getMetadataValue(PgeTaskMetadataKeys.CONFIG_FILE_PATH),
-                    pgeMetadata.getMetadata()), null, pgeConfig, pgeMetadata);
+                    .getMetadata(PgeTaskMetadataKeys.CONFIG_FILE_PATH),
+                    pgeMetadata.asMetadata()), null, pgeConfig, pgeMetadata);
             return pgeConfig;
         } catch (Exception e) {
             e.printStackTrace();
             throw new IOException("Failed to build PgeConfig : "
                     + e.getMessage());
         }
-        
     }
     
     private void buildImports(String configFile, String namespace, PgeConfig pgeConfig, PgeMetadata pgeMetadata) throws MalformedURLException, Exception {
@@ -87,9 +112,9 @@ public class XmlFilePgeConfigBuilder imp
         for (int i = 0; i < nodeList.getLength(); i++) {
             String curImportNS = this.fillIn(((Element) nodeList.item(i))
                     .getAttribute(NAMESPACE_ATTR), pgeMetadata
-                    .getMetadata());
+                    .asMetadata());
             String file = this.fillIn(((Element) nodeList.item(i))
-                    .getAttribute(FILE_ATTR), pgeMetadata.getMetadata());
+                    .getAttribute(FILE_ATTR), pgeMetadata.asMetadata());
             if (!file.startsWith(File.separator))
                 file = new File(configFile).getParent()
                         + File.separator + file;
@@ -99,7 +124,6 @@ public class XmlFilePgeConfigBuilder imp
         // load base config file
         LOG.log(Level.INFO, "Loading PgeConfig file '" + configFile + "'");
         this.build(root, namespace, pgeConfig, pgeMetadata);
-
     }
 
     private void build(Element root, String namespace, PgeConfig pgeConfig,
@@ -109,9 +133,9 @@ public class XmlFilePgeConfigBuilder imp
         PgeMetadata localPgeMetadata = this.getCustomMetadata((Element) root
                 .getElementsByTagName(CUSTOM_METADATA_TAG).item(0), pgeMetadata);
         PgeMetadata curPgeMetadata = new PgeMetadata();
-        curPgeMetadata.addPgeMetadata(pgeMetadata);
-        curPgeMetadata.addPgeMetadata(localPgeMetadata);
-        Metadata curMetadata = curPgeMetadata.getMetadata();
+        curPgeMetadata.replaceMetadata(pgeMetadata);
+        curPgeMetadata.replaceMetadata(localPgeMetadata);
+        Metadata curMetadata = curPgeMetadata.asMetadata();
         
         // load dynamic config file info
         List<DynamicConfigFile> configFileList = this.getDynConfigFile(
@@ -139,16 +163,16 @@ public class XmlFilePgeConfigBuilder imp
             pgeConfig.addOuputDirAndExpressions(outputDir);
         
         // add local pge metadata to global pge metadata with given namespace
-        pgeMetadata.addPgeMetadata(localPgeMetadata, namespace);
+        pgeMetadata.replaceMetadata(localPgeMetadata, namespace);
     }
 
     private PgeMetadata getCustomMetadata(Element customMetadataElem, PgeMetadata pgeMetadata)
             throws MalformedURLException, Exception {
     	PgeMetadata localPgeMetadata = new PgeMetadata();
     	PgeMetadata curPgeMetadata = new PgeMetadata();
-    	curPgeMetadata.addPgeMetadata(pgeMetadata);
+    	curPgeMetadata.replaceMetadata(pgeMetadata);
     	Metadata curPlusLocalMetadata = new Metadata();
-    	curPlusLocalMetadata.addMetadata(curPgeMetadata.getMetadata().getHashtable());
+    	curPlusLocalMetadata.addMetadata(curPgeMetadata.asMetadata().getHashtable());
     	
         if (customMetadataElem != null) {
             NodeList customMetadataList = customMetadataElem
@@ -184,14 +208,14 @@ public class XmlFilePgeConfigBuilder imp
                     	valList.addAll(Arrays.asList((val + ",").split(",")));
                     else 
                     	valList.add(val);
-                    localPgeMetadata.addCustomMetadata(key, valList);
-                    curPgeMetadata.addCustomMetadata(key, valList);
+                    localPgeMetadata.replaceMetadata(key, valList);
+                    curPgeMetadata.replaceMetadata(key, valList);
                 }
                 if (metadataElement.getAttribute(WORKFLOW_MET_ATTR)
                         .toLowerCase().equals("true"))
-                	localPgeMetadata.addWorkflowMetadataKey(key);
+                	localPgeMetadata.markAsDynamicMetadataKey(key);
                 
-                curPlusLocalMetadata.replaceMetadata(key, curPgeMetadata.getMetadataValues(key));
+                curPlusLocalMetadata.replaceMetadata(key, curPgeMetadata.getAllMetadata(key));
             }
         }
         return localPgeMetadata;

Modified: oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/metadata/PgeMetadata.java
URL: http://svn.apache.org/viewvc/oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/metadata/PgeMetadata.java?rev=1299921&r1=1299920&r2=1299921&view=diff
==============================================================================
--- oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/metadata/PgeMetadata.java (original)
+++ oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/metadata/PgeMetadata.java Mon Mar 12 22:57:22 2012
@@ -14,221 +14,447 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.oodt.cas.pge.metadata;
 
 //JDK imports
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
+import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
-import java.util.Vector;
-import java.util.Map.Entry;
+
+//Apache imports
+import org.apache.commons.lang.Validate;
 
 //OODT imports
 import org.apache.oodt.cas.metadata.Metadata;
-import org.apache.oodt.cas.workflow.structs.WorkflowTaskConfiguration;
+
+//Google imports
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 
 /**
+ * A wrapper class to act as a facade interface to all the different
+ * {@link Metadata} sources given to a PGE.
  * 
- * @author bfoster
- * @version $Revision$
+ * NOTE: 2 ways to update DYNAMIC metadata: 1) Create a key link to a DYNAMIC metadata
+ * key, then change the value of the key link or 2) add metadata then mark the
+ * key as dynamic and commit it.
  * 
- * <p>
- * A wrapper class to act as a facade interface to all the different
- * {@link Metadata} sources given to a PGE
- * </p>.
+ * @author bfoster (Brian Foster)
  */
 public class PgeMetadata {
 
-    private Metadata dynMetadata, staticMetadata, customMetadata;
-
-    private WorkflowTaskConfiguration staticConfig;
-
-    public static final int DYN = 1;
-
-    public static final int STATIC = 2;
-
-    public static final int CUSTOM = 3;
-
-    private static int[] defaultMetadataQueryOrder = new int[] {
-            PgeMetadata.STATIC, PgeMetadata.DYN, PgeMetadata.CUSTOM };
-
-    private static int[] defaultMetadataCombineOrder = new int[] {
-            PgeMetadata.CUSTOM, PgeMetadata.DYN, PgeMetadata.STATIC };
-
-    private HashMap<String, String> keyLinkMap;
-    
-    private Set<String> workflowMetKeys;
-    
-    public PgeMetadata() throws Exception {
-    	this(null, null);
-    }
-    
-    public PgeMetadata(Metadata dynamicMetadata,
-            WorkflowTaskConfiguration staticConfig) throws Exception {
-        this.keyLinkMap = new HashMap<String, String>();
-        this.workflowMetKeys = new HashSet<String>();
-        this.customMetadata = new Metadata();
-        this.dynMetadata = dynamicMetadata != null ? dynamicMetadata : new Metadata();
-		this.staticMetadata = staticConfig != null ? this.convertToMetadata(staticConfig, dynamicMetadata) : new Metadata();
-		this.staticConfig = staticConfig != null ? staticConfig : new WorkflowTaskConfiguration();
-	}
-
-	private Metadata convertToMetadata(WorkflowTaskConfiguration config,
-			Metadata dynMetadata) throws Exception {
-		Metadata metadata = new Metadata();
-		for (Entry entry : config.getProperties().entrySet())
-			metadata.replaceMetadata((String) entry.getKey(), (String) entry.getValue());
-		return metadata;
-	}
-
-    public void addWorkflowMetadataKey(String key) {
-        this.workflowMetKeys.add(key);
-    }
-    
-    public void addPgeMetadata(PgeMetadata pgeMetadata) {
-    	this.addPgeMetadata(pgeMetadata, null);
-    }
-    
-    //Namespaces custom metadata, keyLinkMap keys (if key-ref exists in custom metadata or keyLinkMap) and, workflowMetKeys (if they exist in custom metadata or keyLinkMap)
-    public void addPgeMetadata(PgeMetadata pgeMetadata, String namespace) {
-    	this.dynMetadata.addMetadata(pgeMetadata.dynMetadata.getHashtable(), true);
-    	this.staticMetadata.addMetadata(pgeMetadata.staticMetadata.getHashtable(), true);
-    	this.staticConfig.getProperties().putAll(pgeMetadata.staticConfig.getProperties());
-    	for (Object key : pgeMetadata.customMetadata.getHashtable().keySet())
-    		this.addCustomMetadata(namespace != null ? namespace + ":" + (String) key : (String) key, pgeMetadata.customMetadata.getAllMetadata((String) key));
-    	for (String key : pgeMetadata.keyLinkMap.keySet()) {
-    		String value = pgeMetadata.keyLinkMap.get(key);
-    		if (namespace != null && (pgeMetadata.customMetadata.containsKey(value) || pgeMetadata.keyLinkMap.containsKey(value)))
-    			value = namespace + ":" + value;
-    		this.linkKey(namespace != null ? namespace + ":" + key : key, value);
-    	}
-    	for (String key : pgeMetadata.workflowMetKeys)
-    		this.addWorkflowMetadataKey((namespace != null && (pgeMetadata.customMetadata.containsKey(key) || pgeMetadata.keyLinkMap.containsKey(key))) ? namespace + ":" + key : key);
-    }
-    
-    public void commitWorkflowMetadataKeys() {
-        for (String key : this.workflowMetKeys) 
-            this.dynMetadata.replaceMetadata(key, this.getMetadataValues(key));
-        this.workflowMetKeys.clear();
-    }
-    
-    public void linkKey(String keyName, String linkToKeyName) {
-    	this.customMetadata.removeMetadata(keyName);
-        this.keyLinkMap.put(keyName, linkToKeyName);
-    }
-    
-    /**
-     * Included for backwards compatibility with oco-pge
-     * 
-     * @return
-     */
-    @Deprecated
-    public WorkflowTaskConfiguration getWorkflowTaskConfiguration() {
-        return this.staticConfig;
-    }
-
-    /**
-     * Included for backwards compatibility with oco-pge
-     * 
-     * @return
-     */
-    @Deprecated
-    public Metadata getDynamicMetadata() {
-        return this.dynMetadata;
-    }
-
-    public void addDynamicMetadata(String name, String value) {
-        this.dynMetadata.replaceMetadata(name, value);
-    }
-
-    public void addDynamicMetadata(String name, List<String> values) {
-        this.dynMetadata.replaceMetadata(name, values);
-    }
-
-    public void addCustomMetadata(String name, String value) {
-    	this.keyLinkMap.remove(name);
-        this.customMetadata.replaceMetadata(name, value);
-    }
-
-    public void addCustomMetadata(Metadata metadata) {
-    	for (Object key : metadata.getHashtable().keySet())
-    		this.addCustomMetadata((String) key, metadata.getAllMetadata((String) key));
-    }
-    
-    public void addCustomMetadata(String name, List<String> values) {
-    	this.keyLinkMap.remove(name);
-        this.customMetadata.replaceMetadata(name, values);
-    }
-
-    // add in order which you want metadata added (will return a copy)
-    public Metadata getMetadata(int... types) {
-        if (types.length < 1)
-            types = defaultMetadataCombineOrder;
-        Metadata combinedMetadata = new Metadata();
-        for (int type : types) {
-            switch (type) {
-            case DYN:
-                combinedMetadata.addMetadata(this.dynMetadata.getHashtable(),
-                        true);
-                break;
+   public enum Type {
+      STATIC, DYNAMIC, LOCAL;
+   }
+   public static final List<Type> DEFAULT_PRECENDENCE_HIERARCHY = Lists
+         .newArrayList(Type.DYNAMIC, Type.LOCAL, Type.STATIC);
+
+   private final Metadata staticMetadata;
+   private final Metadata dynamicMetadata;
+   private final Metadata localMetadata;
+
+   private final Map<String, String> keyLinkMap;
+   private final Set<String> markedAsDynamicMetKeys;
+
+   public PgeMetadata() {
+      keyLinkMap = Maps.newHashMap();
+      markedAsDynamicMetKeys = Sets.newHashSet();
+      staticMetadata = new Metadata();
+      dynamicMetadata = new Metadata();
+      localMetadata = new Metadata();
+   }
+
+   public PgeMetadata(PgeMetadata pgeMetadata) {
+      this();
+
+      Validate.notNull(pgeMetadata, "pgeMetadata cannot be null");
+
+      replaceMetadata(pgeMetadata);
+   }
+
+   public PgeMetadata(Metadata staticMetadata, Metadata dynamicMetadata) {
+      this();
+
+      Validate.notNull(staticMetadata, "staticMetadata cannot be null");
+      Validate.notNull(dynamicMetadata, "dynamicMetadata cannot be null");
+
+      this.staticMetadata.replaceMetadata(staticMetadata);
+      this.dynamicMetadata.replaceMetadata(dynamicMetadata);
+   }
+
+   /**
+    * Replaces or creates this {@link PgeMetadata}'s metadata with given
+    * {@link PgeMetadata}'s metadata. Also adds in the list of given
+    * {@link PgeMetadata}'s LOCAL metadata marked for promotion DYNAMIC
+    * metadata and list of key links.
+    * 
+    * @param pgeMetadata
+    *           A {@link PgeMetadata} whose metadata and key links will be added
+    *           to this {@link PgeMetadata}'s metadata and key links.
+    */
+   public void replaceMetadata(PgeMetadata pgeMetadata) {
+      Validate.notNull(pgeMetadata, "pgeMetadata cannot be null");
+
+      staticMetadata.replaceMetadata(pgeMetadata.staticMetadata);
+      dynamicMetadata.replaceMetadata(pgeMetadata.dynamicMetadata);
+      localMetadata.replaceMetadata(pgeMetadata.localMetadata);
+
+      keyLinkMap.putAll(pgeMetadata.keyLinkMap);
+      markedAsDynamicMetKeys.addAll(pgeMetadata.markedAsDynamicMetKeys);
+   }
+
+   /**
+    * Replaces or creates this {@link PgeMetadata}'s metadata with given
+    * {@link PgeMetadata}'s metadata. The provided "group" will be used to
+    * namespace the given {@link PgeMetadata}'s LOCAL metadata when add to this
+    * {@link PgeMetadata}'s LOCAL metadata. It will also namespace given
+    * {@link PgeMetadata}'s key links before adding then to this
+    * {@link PgeMetadata}'s key links. Also add in the list of given
+    * {@link PgeMetadata}'s LOCAL metadata marked for promotion DYNAMIC
+    * metadata.
+    * 
+    * @param pgeMetadata
+    *           A {@link PgeMetadata} whose metadata and key links will be added
+    *           to this {@link PgeMetadata}'s metadata and key links.
+    * @param group
+    *           The namespace which will be used to namespace given
+    *           {@link PgeMetadata}'s LOCAL metadata and key links before being
+    *           added to this {@link PgeMetadata}'s LOCAL metadata and key
+    *           links.
+    */
+   public void replaceMetadata(PgeMetadata pgeMetadata, String group) {
+      Validate.notNull(pgeMetadata, "pgeMetadata cannot be null");
+      Validate.notNull(group, "group cannot be null");
+
+      staticMetadata.replaceMetadata(pgeMetadata.staticMetadata);
+      dynamicMetadata.replaceMetadata(pgeMetadata.dynamicMetadata);
+      localMetadata.replaceMetadata(group, pgeMetadata.localMetadata);
+
+      // Namespace link keys that point to either importing
+      // metadata's local key or link key.
+      for (String keyLink : pgeMetadata.keyLinkMap.keySet()) {
+         String key = pgeMetadata.keyLinkMap.get(keyLink);
+         // Check if key is was local key or a link key
+         if (pgeMetadata.localMetadata.containsKey(key)
+               || pgeMetadata.keyLinkMap.containsKey(key)) {
+            key = group + "/" + key;
+         }
+         linkKey(group + "/" + keyLink, key);
+      }
+
+      // Namespace workflow keys that point to either importing
+      // metadata's local key or link key.
+      for (String key : pgeMetadata.markedAsDynamicMetKeys) {
+         if (pgeMetadata.localMetadata.containsKey(key)
+               || pgeMetadata.keyLinkMap.containsKey(key)) {
+            key = group + "/" + key;
+         }
+         markAsDynamicMetadataKey(key);
+      }
+   }
+
+   /**
+    * Use to mark LOCAL keys which should be moved into DYNAMIC metadata when
+    * {@link #commitMarkedDynamicMetadataKeys(String...)} is invoked. If no 
+    * args are specified then all LOCAL metadata is marked for move to
+    * DYNAMIC metadata.
+    * 
+    * @param keys
+    *           Keys to mark as to be made DYNAMIC, otherwise if no keys then
+    *           all LOCAL metadata keys are mark for move to DYNAMIC.
+    */
+   public void markAsDynamicMetadataKey(String... keys) {
+      List<String> markedKeys = Lists.newArrayList(keys);
+      if (markedKeys.isEmpty()) {
+         markedKeys.addAll(localMetadata.getAllKeys());
+      }
+      markedAsDynamicMetKeys.addAll(markedKeys);
+   }
+
+   /**
+    * Use to commit marked LOCAL keys to DYNAMIC keys. Specify a list of keys
+    * only if you want to limit the keys which get committed, otherwise all
+    * marked keys will be moved into DYNAMIC metadata.
+    * 
+    * @param keys
+    *           The list of marked LOCAL metadata keys which should be moved
+    *           into DYNAMIC metadata. If no keys are specified then all marked
+    *           keys are moved.
+    */
+   public void commitMarkedDynamicMetadataKeys(String... keys) {
+      Set<String> commitKeys = Sets.newHashSet(keys);
+      if (commitKeys.isEmpty()) {
+         commitKeys.addAll(markedAsDynamicMetKeys);
+      } else {
+         commitKeys.retainAll(markedAsDynamicMetKeys);
+      }
+      for (String key : commitKeys) {
+         dynamicMetadata.replaceMetadata(key,
+               localMetadata.getAllMetadata(resolveKey(key)));
+         localMetadata.removeMetadata(key);
+         markedAsDynamicMetKeys.remove(key);
+      }
+   }
+
+   @VisibleForTesting
+   protected Set<String> getMarkedAsDynamicMetadataKeys() {
+      return Collections.unmodifiableSet(markedAsDynamicMetKeys);
+   }
+
+   /**
+    * Create a key which is a link to another key, such that if you get the
+    * metadata values for the created link it will return the current metadata
+    * values of the key it was linked to. NOTE: if the key's metadata values
+    * change, then the metadata values for the link key will also be the changed
+    * values. If you want to create a key which holds the current value of a
+    * key, then create a new metadata key.
+    * 
+    * @param keyLink
+    *           The name of the link key you wish to create.
+    * @param key
+    *           The key you which to link to (may also be a key link)
+    */
+   public void linkKey(String keyLink, String key) {
+      Validate.notNull(keyLink, "keyLink cannot be null");
+      Validate.notNull(key, "key cannot be null");
+
+      localMetadata.removeMetadata(keyLink);
+      keyLinkMap.put(keyLink, key);
+   }
+
+   /**
+    * Removes a key link reference. The key which the key link was linked to
+    * remains unchanged.
+    * 
+    * @param keyLink
+    *           The key link which you wish to destroy.
+    */
+   public void unlinkKey(String keyLink) {
+      Validate.notNull(keyLink, "keyLink cannot be null");
+
+      keyLinkMap.remove(keyLink);
+   }
+
+   /**
+    * Check if the given key name is a key link.
+    * 
+    * @param key
+    *           The key name in question.
+    * @return True is the given key name is a key link, false if key name is an
+    *         actual key.
+    */
+   public boolean isLink(String key) {
+      Validate.notNull(key, "key cannot be null");
+
+      return keyLinkMap.containsKey(key);
+   }
+
+   /**
+    * Find the actual key whose value will be returned for the given key. If the
+    * given key is a key (not a key link) then the given key will just be
+    * returned, otherwise it will trace through key link mapping to find the key
+    * which the given key link points to.
+    * 
+    * @param key
+    *           The name of a key or key link.
+    * @return The key whose value will be returned for the given key or key
+    *         link.
+    */
+   public String resolveKey(String key) {
+      Validate.notNull(key, "key cannot be null");
+
+      while (keyLinkMap.containsKey(key)) {
+         key = keyLinkMap.get(key);
+      }
+      return key;
+   }
+
+   /**
+    * Determines the path by which the given key (if it is a key link) links to
+    * the key whose value it will return. If the given key is a key link and
+    * points to a key then the returning {@link List} will be of size 1 and will
+    * contain just that key. However, if the given key is a key link which
+    * points to another key link then the returning {@link List} will be greater
+    * than 1 (will depend on how many key links are connected before they actual
+    * point to a key. If the given key is a key, then the returning {@link List}
+    * will be empty.
+    * 
+    * @param key
+    *           The path to the key whose value will be returned for the give
+    *           key.
+    * @return A key path {@link List}.
+    */
+   public List<String> getReferenceKeyPath(String key) {
+      Validate.notNull(key, "key cannot be null");
+
+      List<String> keyPath = Lists.newArrayList();
+      while (keyLinkMap.containsKey(key))
+         keyPath.add(key = keyLinkMap.get(key));
+      return keyPath;
+   }
+
+   /**
+    * Replace the given key's value with the given value. If the given key is a
+    * key link, then it will update the value of the key it is linked to if that
+    * key is DYNAMIC or LOCAL. If given key is a key link and it links to a
+    * STATIC key, then a new LOCAL key will be create.
+    * 
+    * @param key
+    *           The key or key link for whose value should be replaced.
+    * @param value
+    *           The value to give the given key. Will replace any existing value
+    *           or will be the value of a newly created LOCAL key.
+    */
+   public void replaceMetadata(String key, String value) {
+      Validate.notNull(key, "key cannot be null");
+      Validate.notNull(value, "value cannot be null");
+
+      String resolveKey = resolveKey(key);
+      // If key is a key link which points to a DYNAMIC key then update the
+      // DYNAMIC key's value.
+      if (keyLinkMap.containsKey(key)
+            && dynamicMetadata.containsKey(resolveKey)) {
+         dynamicMetadata.replaceMetadata(resolveKey, value);
+      } else {
+         localMetadata.replaceMetadata(resolveKey, value);
+      }
+   }
+
+   /**
+    * Replace all key values with the given key values in the provided
+    * {@link Metadata}. If the key does not exist it will be created.
+    * 
+    * @param metadata
+    *           {@link Metadata} to replace or create.
+    */
+   public void replaceMetadata(Metadata metadata) {
+      Validate.notNull(metadata, "metadata cannot be null");
+
+      for (String key : metadata.getAllKeys()) {
+         replaceMetadata(key, metadata.getAllMetadata(key));
+      }
+   }
+
+   /**
+    * Replace the given key's values with the given values. If the given key is
+    * a key link, then it will update the values of the key it is linked to if
+    * that key is DYNAMIC or LOCAL. If given key is a key link and it links to a
+    * STATIC key, then a new LOCAL key will be create.
+    * 
+    * @param key
+    *           The key or key link for whose values should be replaced.
+    * @param values
+    *           The values to give the given key. Will replace any existing
+    *           values or will be the values of a newly created LOCAL key.
+    */
+   public void replaceMetadata(String key, List<String> values) {
+      Validate.notNull(key, "key cannot be null");
+      Validate.notNull(values, "values cannot be null");
+
+      String resolveKey = resolveKey(key);
+      if (keyLinkMap.containsKey(key) && dynamicMetadata.containsKey(resolveKey)) {
+         dynamicMetadata.replaceMetadata(resolveKey, values);
+      } else {
+         localMetadata.replaceMetadata(resolveKey, values);
+      }
+   }
+
+   /**
+    * Combines STATIC, DYNAMIC, and LOCAL metadata into one metadata object. You
+    * can restrict which metadata you want combined and change the order in
+    * which combining takes place by specifying Type arguments in the order you
+    * which precedence to be observed. For example, if you perform the
+    * following: pgeMetadata.asMetadata(LOCAL, STATIC) then only LOCAL and
+    * STATIC metadata will be combined and LOCAL metadata will trump STATIC
+    * metadata if they both contain the same key. If no arguments are specified
+    * then DEFAULT_PRECENDENCE_HIERARCHY is used.
+    * 
+    * @param types
+    *           The Type hierarchy you which to use when metadata is combined,
+    *           if no args then DEFAULT_PRECENDENCE_HIERARCHY is used.
+    * @return Combined metadata.
+    */
+   public Metadata asMetadata(Type... types) {
+      List<Type> conbineOrder = Lists.newArrayList(types);
+      if (conbineOrder.isEmpty()) {
+         conbineOrder.addAll(DEFAULT_PRECENDENCE_HIERARCHY);
+      }
+      Collections.reverse(conbineOrder);
+
+      Metadata combinedMetadata = new Metadata();
+      for (Type type : conbineOrder) {
+         switch (type) {
+            case DYNAMIC:
+               combinedMetadata.replaceMetadata(dynamicMetadata);
+               break;
             case STATIC:
-                combinedMetadata.addMetadata(
-                        this.staticMetadata.getHashtable(), true);
-                break;
-            case CUSTOM:
-                combinedMetadata.addMetadata(
-                        this.customMetadata.getHashtable(), true);
-                for (Iterator<String> iter = this.keyLinkMap.keySet().iterator(); iter.hasNext(); ) {
-                    String key = iter.next();
-                    combinedMetadata.replaceMetadata(key, this.getMetadataValues(key));
-                }
-                break;
-            }
-        }
-        return combinedMetadata;
-    }
-
-    public List<String> getMetadataValues(String name, int... types) {
-        if (types.length < 1)
-            types = defaultMetadataQueryOrder;
-        String useKeyName = this.resolveKey(name);
-        for (int type : types) {
-            List<String> value = null;
-            switch (type) {
-            case DYN:
-                if ((value = this.dynMetadata.getAllMetadata(useKeyName)) != null)
-                    return value;
-                break;
+               combinedMetadata.replaceMetadata(staticMetadata);
+               break;
+            case LOCAL:
+               combinedMetadata.replaceMetadata(localMetadata);
+               for (String key : keyLinkMap.keySet()) {
+                  combinedMetadata.replaceMetadata(key, getAllMetadata(key));
+               }
+               break;
+         }
+      }
+      return combinedMetadata;
+   }
+
+   /**
+    * Get metadata values for given key. If Types are specified then it provides
+    * the precedence order in which to search for the key. If no Type args are
+    * specified then DEFAULT_PRECENDENCE_HIERARCHY will be used. For example if
+    * you pass in Type args: STATIC, LOCAL then STATIC metadata will first be
+    * checked for the key and if it contains it, then it will return the found
+    * value, otherwise it will then check LOCAL metadata for the key and if it
+    * finds the value it will return it, otherwise null.
+    * 
+    * @param key
+    *           The key for whose metadata values should be returned.
+    * @param types
+    *           The type hierarchy which should be used, if no Types specified
+    *           DEFAULT_PRECENDENCE_HIERARCHY will be used.
+    * @return Metadata values for given key.
+    */
+   public List<String> getAllMetadata(String key, Type... types) {
+      List<Type> queryOrder = Lists.newArrayList(types);
+      if (queryOrder.isEmpty()) {
+         queryOrder.addAll(DEFAULT_PRECENDENCE_HIERARCHY);
+      }
+
+      String useKey = resolveKey(key);
+      for (Type type : queryOrder) {
+         switch (type) {
+            case DYNAMIC:
+               if (dynamicMetadata.containsKey(useKey)) {
+                  return dynamicMetadata.getAllMetadata(useKey);
+               }
+               break;
             case STATIC:
-                if ((value = this.staticMetadata.getAllMetadata(useKeyName)) != null)
-                    return value;
-                break;
-            case CUSTOM:
-                if ((value = this.customMetadata.getAllMetadata(useKeyName)) != null)
-                    return value;
-                break;
-            }
-        }
-        return new Vector<String>();
-    }
-
-    public String getMetadataValue(String name, int... types) {
-        List<String> values = this.getMetadataValues(name, types);
-        if (values.size() > 0)
-            return values.get(0);
-        else
-            return null;
-    }
-    
-    public String resolveKey(String keyName) {
-        String useKeyName = keyName;
-        while(this.keyLinkMap.containsKey(useKeyName))
-            useKeyName = this.keyLinkMap.get(useKeyName);
-        return useKeyName;
-    }
-
+               if (staticMetadata.containsKey(useKey)) {
+                  return staticMetadata.getAllMetadata(useKey);
+               }
+               break;
+            case LOCAL:
+               if (localMetadata.containsKey(useKey)) {
+                  return localMetadata.getAllMetadata(useKey);
+               }
+               break;
+         }
+      }
+      return Lists.newArrayList();
+   }
+
+   /**
+    * Returns the first value returned by {@link #getAllMetadata(String, Type...)}, if it returns
+    * null then this method will also return null.
+    */
+   public String getMetadata(String key, Type... types) {
+      List<String> values = getAllMetadata(key, types);
+      return (values.size() > 0) ? values.get(0) : null;
+   }
 }

Modified: oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/writers/PcsMetFileWriter.java
URL: http://svn.apache.org/viewvc/oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/writers/PcsMetFileWriter.java?rev=1299921&r1=1299920&r2=1299921&view=diff
==============================================================================
--- oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/writers/PcsMetFileWriter.java (original)
+++ oodt/trunk/pge/src/main/java/org/apache/oodt/cas/pge/writers/PcsMetFileWriter.java Mon Mar 12 22:57:22 2012
@@ -14,8 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.oodt.cas.pge.writers;
 
 //OODT imports
@@ -27,13 +25,9 @@ import org.apache.oodt.cas.metadata.Meta
 import java.io.File;
 
 /**
- * 
- * @author bfoster
- * @version $Revision$
- * 
- * <p>
- * Writes a PCS metadata file for the give data file
- * </p>.
+ * Writes a PCS metadata file for the give data file.
+ *
+ * @author bfoster (Brian Foster)
  */
 public abstract class PcsMetFileWriter {
 
@@ -42,7 +36,7 @@ public abstract class PcsMetFileWriter {
     public Metadata getMetadataForFile(File sciPgeCreatedDataFile,
             PgeMetadata pgeMetadata, Object... customArgs) throws Exception {
         try {
-            Metadata inputMetadata = pgeMetadata.getMetadata();
+            Metadata inputMetadata = pgeMetadata.asMetadata();
 
             inputMetadata.replaceMetadata(CoreMetKeys.FILENAME,
                     sciPgeCreatedDataFile.getName());

Added: oodt/trunk/pge/src/test/org/apache/oodt/cas/pge/metadata/TestPgeMetadata.java
URL: http://svn.apache.org/viewvc/oodt/trunk/pge/src/test/org/apache/oodt/cas/pge/metadata/TestPgeMetadata.java?rev=1299921&view=auto
==============================================================================
--- oodt/trunk/pge/src/test/org/apache/oodt/cas/pge/metadata/TestPgeMetadata.java (added)
+++ oodt/trunk/pge/src/test/org/apache/oodt/cas/pge/metadata/TestPgeMetadata.java Mon Mar 12 22:57:22 2012
@@ -0,0 +1,156 @@
+/*
+ * 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.oodt.cas.pge.metadata;
+
+//OODT imports
+import org.apache.oodt.cas.metadata.Metadata;
+import org.apache.oodt.cas.pge.metadata.PgeMetadata.Type;
+
+//Google imports
+import com.google.common.collect.Lists;
+
+//JUnit imports
+import junit.framework.TestCase;
+
+/**
+ * Test class for {@link PgeMetadata}.
+ * 
+ * @author bfoster (Brian Foster)
+ */
+public class TestPgeMetadata extends TestCase {
+
+   public void testReplacePgeMetadata() {
+      Metadata staticMetadata = new Metadata();
+      staticMetadata.addMetadata("key1", "staticValue1");
+      staticMetadata.addMetadata("key2", "staticValue2");
+      staticMetadata.addMetadata("key3", "staticValue3");
+
+      Metadata dynamicMetadata = new Metadata();
+      dynamicMetadata.addMetadata("key2", "dynValue2");
+
+      PgeMetadata pgeMet = new PgeMetadata(staticMetadata, dynamicMetadata);
+      PgeMetadata pgeMet2 = new PgeMetadata(staticMetadata, dynamicMetadata);
+      pgeMet2.replaceMetadata("key4", "localValue4");
+      pgeMet2.linkKey("keyLink1", "key1");
+      pgeMet2.markAsDynamicMetadataKey("key4");
+      pgeMet.replaceMetadata(pgeMet2, "test");
+
+      assertEquals("staticValue1", pgeMet.getMetadata("key1"));
+      assertEquals("dynValue2", pgeMet.getMetadata("key2"));
+      assertEquals("staticValue2",
+            pgeMet.getMetadata("key2", PgeMetadata.Type.STATIC));
+      assertEquals("staticValue3", pgeMet.getMetadata("key3"));
+      assertNull(pgeMet.getMetadata("key4"));
+
+      assertNull(pgeMet.getMetadata("test/key1"));
+      assertNull(pgeMet.getMetadata("test/key2"));
+      assertNull(pgeMet.getMetadata("test/key2", PgeMetadata.Type.STATIC));
+      assertNull(pgeMet.getMetadata("test/key3"));
+      assertEquals("localValue4", pgeMet.getMetadata("test/key4"));
+
+      assertNull(pgeMet.getMetadata("keyLink1"));
+      assertEquals(pgeMet.getMetadata("key1"),
+            pgeMet.getMetadata("test/keyLink1"));
+      assertEquals(1, pgeMet.getMarkedAsDynamicMetadataKeys().size());
+      assertEquals("test/key4", pgeMet.getMarkedAsDynamicMetadataKeys()
+            .iterator().next());
+   }
+
+   public void testChangingDynamicMetadata() {
+      PgeMetadata pgeMet = new PgeMetadata();
+
+      assertTrue(pgeMet.asMetadata(Type.DYNAMIC).getAllKeys().isEmpty());
+      pgeMet.replaceMetadata("key1", "value1");
+      assertTrue(pgeMet.asMetadata(Type.DYNAMIC).getAllKeys().isEmpty());
+      pgeMet.markAsDynamicMetadataKey();
+      assertTrue(pgeMet.asMetadata(Type.DYNAMIC).getAllKeys().isEmpty());
+      pgeMet.commitMarkedDynamicMetadataKeys();
+      assertEquals(1, pgeMet.asMetadata(Type.DYNAMIC).getAllKeys().size());
+      assertEquals("value1", pgeMet.asMetadata(Type.DYNAMIC)
+            .getMetadata("key1"));
+
+      pgeMet = new PgeMetadata();
+
+      assertTrue(pgeMet.asMetadata(Type.DYNAMIC).getAllKeys().isEmpty());
+      pgeMet.replaceMetadata("key1", "value1");
+      assertTrue(pgeMet.asMetadata(Type.DYNAMIC).getAllKeys().isEmpty());
+      pgeMet.markAsDynamicMetadataKey("key1");
+      assertTrue(pgeMet.asMetadata(Type.DYNAMIC).getAllKeys().isEmpty());
+      pgeMet.commitMarkedDynamicMetadataKeys("key1");
+      assertEquals(1, pgeMet.asMetadata(Type.DYNAMIC).getAllKeys().size());
+      assertEquals("value1", pgeMet.asMetadata(Type.DYNAMIC)
+            .getMetadata("key1"));
+
+      pgeMet.linkKey("keyLink1", "key1");
+      pgeMet.replaceMetadata("keyLink1", "newValue");
+      assertEquals("newValue",
+            pgeMet.asMetadata(Type.DYNAMIC).getMetadata("key1"));
+   }
+
+   public void testLinking() {
+      PgeMetadata pgeMet = new PgeMetadata();
+
+      pgeMet.replaceMetadata("key1", "value1");
+      pgeMet.linkKey("keyLink1", "key1");
+      pgeMet.linkKey("keyLink2", "keyLink1");
+      pgeMet.linkKey("keyLink3", "keyLink1");
+
+      assertEquals("value1", pgeMet.getMetadata("key1"));
+      assertEquals("value1", pgeMet.getMetadata("keyLink1"));
+      assertEquals("value1", pgeMet.getMetadata("keyLink2"));
+      assertEquals("value1", pgeMet.getMetadata("keyLink3"));
+
+      pgeMet.replaceMetadata("keyLink3", "newValue");
+
+      assertEquals("newValue", pgeMet.getMetadata("key1"));
+      assertEquals("newValue", pgeMet.getMetadata("keyLink1"));
+      assertEquals("newValue", pgeMet.getMetadata("keyLink2"));
+      assertEquals("newValue", pgeMet.getMetadata("keyLink3"));
+
+      assertEquals(Lists.newArrayList("keyLink1", "key1"),
+            pgeMet.getReferenceKeyPath("keyLink2"));
+      assertEquals(Lists.newArrayList("keyLink1", "key1"),
+            pgeMet.getReferenceKeyPath("keyLink3"));
+
+      pgeMet.replaceMetadata("key1", "value1");
+
+      assertEquals("value1", pgeMet.getMetadata("key1"));
+      assertEquals("value1", pgeMet.getMetadata("keyLink1"));
+      assertEquals("value1", pgeMet.getMetadata("keyLink2"));
+      assertEquals("value1", pgeMet.getMetadata("keyLink3"));
+
+      pgeMet.unlinkKey("keyLink1");
+
+      assertEquals("value1", pgeMet.getMetadata("key1"));
+      assertNull(pgeMet.getMetadata("keyLink1"));
+      assertNull(pgeMet.getMetadata("keyLink2"));
+      assertNull(pgeMet.getMetadata("keyLink3"));
+
+      pgeMet.linkKey("keyLink1", "key1");
+
+      assertEquals("value1", pgeMet.getMetadata("key1"));
+      assertEquals("value1", pgeMet.getMetadata("keyLink1"));
+      assertEquals("value1", pgeMet.getMetadata("keyLink2"));
+      assertEquals("value1", pgeMet.getMetadata("keyLink3"));
+
+      assertEquals("key1", pgeMet.resolveKey("key1"));
+      assertEquals("key1", pgeMet.resolveKey("keyLink1"));
+      assertEquals("key1", pgeMet.resolveKey("keyLink2"));
+      assertEquals("key1", pgeMet.resolveKey("keyLink3"));
+      assertEquals("keyLink4", pgeMet.resolveKey("keyLink4"));
+   }
+}

Propchange: oodt/trunk/pge/src/test/org/apache/oodt/cas/pge/metadata/TestPgeMetadata.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain