implemented transformation test

pull/94/head
Sandro La Bruzzo 3 years ago
parent 73d772a4b4
commit 0634674add

@ -27,7 +27,7 @@ object GenerateDataciteDatasetSpark {
val isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl)
val vocabularies = VocabularyGroup.loadVocsFromIS(isLookupService)
log.info(s"vocabulary size is ${vocabularies.getTerms("dnet:languages").size()}")
val spark: SparkSession = SparkSession.builder().config(conf)
.appName(GenerateDataciteDatasetSpark.getClass.getSimpleName)
.master(master)

@ -24,6 +24,7 @@ import eu.dnetlib.data.mdstore.manager.common.model.MDStoreVersion;
import eu.dnetlib.dhp.aggregation.common.AggregationCounter;
import eu.dnetlib.dhp.aggregation.common.AggregationUtility;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
import eu.dnetlib.dhp.model.mdstore.MetadataRecord;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
@ -60,15 +61,23 @@ public class TransformSparkJobNode {
final String isLookupUrl = parser.get("isLookupUrl");
log.info(String.format("isLookupUrl: %s", isLookupUrl));
final String dateOfTransformation = parser.get("dateOfTransformation");
log.info(String.format("dateOfTransformation: %s", dateOfTransformation));
final ISLookUpService isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl);
final VocabularyGroup vocabularies = VocabularyGroup.loadVocsFromIS(isLookupService);
log.info("Retrieved {} vocabularies", vocabularies.vocabularyNames().size());
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> transformRecords(
parser.getObjectMap(), isLookupService, spark, nativeMdStoreVersion.getHdfsPath(),
cleanedMdStoreVersion.getHdfsPath()));
parser.getObjectMap(), isLookupService, spark, nativeMdStoreVersion.getHdfsPath() + "/store",
cleanedMdStoreVersion.getHdfsPath() + "/store"));
}
public static void transformRecords(final Map<String, String> args, final ISLookUpService isLookUpService,
@ -82,7 +91,7 @@ public class TransformSparkJobNode {
final Encoder<MetadataRecord> encoder = Encoders.bean(MetadataRecord.class);
final Dataset<MetadataRecord> mdstoreInput = spark.read().format("parquet").load(inputPath).as(encoder);
final MapFunction<MetadataRecord, MetadataRecord> XSLTTransformationFunction = TransformationFactory
.getTransformationPlugin(args, ct, isLookUpService);
.getTransformationPlugin(args, ct, isLookUpService);
mdstoreInput.map(XSLTTransformationFunction, encoder).write().save(outputPath + "/store");
log.info("Transformed item " + ct.getProcessedItems().count());

@ -18,7 +18,7 @@ import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
public class TransformationFactory {
private static final Logger log = LoggerFactory.getLogger(TransformationFactory.class);
public static final String TRULE_XQUERY = "for $x in collection('/db/DRIVER/TransformationRuleDSResources/TransformationRuleDSResourceType') where $x//RESOURCE_IDENTIFIER/@value = \"%s\" return $x//CODE/text()";
public static final String TRULE_XQUERY = "for $x in collection('/db/DRIVER/TransformationRuleDSResources/TransformationRuleDSResourceType') where $x//RESOURCE_IDENTIFIER/@value = \"%s\" return $x//CODE/*[local-name() =\"stylesheet\"]";
public static MapFunction<MetadataRecord, MetadataRecord> getTransformationPlugin(
final Map<String, String> jobArgument, final AggregationCounter counters, final ISLookUpService isLookupService)
@ -57,7 +57,7 @@ public class TransformationFactory {
private static String queryTransformationRuleFromIS(final String transformationRuleId,
final ISLookUpService isLookUpService) throws Exception {
final String query = String.format(TRULE_XQUERY, transformationRuleId);
log.info("asking query to IS: " + query);
System.out.println("asking query to IS: " + query);
List<String> result = isLookUpService.quickSearchProfile(query);
if (result == null || result.isEmpty())

@ -15,8 +15,5 @@
<name>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
</property>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>

@ -18,12 +18,17 @@
</property>
<property>
<name>transformationPlugin</name>
<value>XSLT_TRANSFORM</value>
<description>The transformation Plugin</description>
</property>
<property>
<name>dateOfTransformation</name>
<description>The timestamp of the transformation date</description>
</property>
<property>
<name>isLookupUrl</name>
<description>The IS lookUp service endopoint</description>
</property>
</parameters>
@ -35,22 +40,36 @@
<action name="BeginRead">
<java>
<configuration>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
<arg>--action</arg><arg>READ_LOCK</arg>
<arg>--mdStoreID</arg><arg>${mdStoreInputId}</arg>
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
<capture-output/>
</java>
<ok to="StartTransaction"/>
<error to="Kill"/>
</action>
<action name="StartTransaction">
<java>
<configuration>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
<arg>--action</arg><arg>NEW_VERSION</arg>
<arg>--mdStoreID</arg><arg>${mdStoreOutputId}</arg>
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
<capture-output/>
</java>
<ok to="TransformJob"/>
<error to="EndReadRollBack"/>
@ -62,7 +81,7 @@
<mode>cluster</mode>
<name>Transform MetadataStore</name>
<class>eu.dnetlib.dhp.transformation.TransformSparkJobNode</class>
<jar>dhp-aggregations-${projectVersion}.jar</jar>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
@ -72,11 +91,12 @@
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--mdstoreInputVersion</arg><arg>${wf:actionData('StartTransaction')['mdStoreVersion']}</arg>
<arg>--mdstoreOutputVersion</arg><arg>${wf:actionData('BeginRead')['mdStoreReadLockVersion']}</arg>
<arg>--mdstoreOutputVersion</arg><arg>${wf:actionData('StartTransaction')['mdStoreVersion']}</arg>
<arg>--mdstoreInputVersion</arg><arg>${wf:actionData('BeginRead')['mdStoreReadLockVersion']}</arg>
<arg>--dateOfTransformation</arg><arg>${dateOfTransformation}</arg>
<arg>--transformationPlugin</arg><arg>${transformationPlugin}</arg>
<arg>--transformationRuleId</arg><arg>${transformationRuleId}</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
</spark>
<ok to="EndRead"/>
<error to="EndReadRollBack"/>
@ -84,6 +104,13 @@
<action name="EndRead">
<java>
<configuration>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
<arg>--action</arg><arg>READ_UNLOCK</arg>
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
@ -96,6 +123,12 @@
<action name="CommitVersion">
<java>
<configuration>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
<arg>--action</arg><arg>COMMIT</arg>
<arg>--namenode</arg><arg>${nameNode}</arg>
@ -108,18 +141,30 @@
<action name="EndReadRollBack">
<java>
<configuration>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
<arg>--action</arg><arg>READ_UNLOCK</arg>
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
<arg>--readMDStoreId</arg><arg>${wf:actionData('BeginRead')['mdStoreReadLockVersion']}</arg>
<capture-output/>
</java>
</java>
<ok to="RollBack"/>
<error to="Kill"/>
</action>
<action name="RollBack">
<java>
<configuration>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
<arg>--action</arg><arg>ROLLBACK</arg>
<arg>--mdStoreVersion</arg><arg>${wf:actionData('StartTransaction')['mdStoreVersion']}</arg>

@ -0,0 +1,197 @@
package eu.dnetlib.dhp.aggregation;
import java.io.File;
import java.io.FileOutputStream;
import java.io.FileReader;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.HashMap;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import eu.dnetlib.dhp.collection.GenerateNativeStoreSparkJob;
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
import eu.dnetlib.dhp.transformation.TransformSparkJobNode;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Text;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoder;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession;
import org.junit.jupiter.api.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.data.mdstore.manager.common.model.MDStoreVersion;
import eu.dnetlib.dhp.model.mdstore.MetadataRecord;
@TestMethodOrder(MethodOrderer.OrderAnnotation.class)
public class AggregationJobTest {
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static SparkSession spark;
private static Path workingDir;
private static Encoder<MetadataRecord> encoder;
private static final String encoding = "XML";
private static final String dateOfCollection = System.currentTimeMillis() + "";
private static final String xpath = "//*[local-name()='header']/*[local-name()='identifier']";
private static String provenance;
private static final Logger log = LoggerFactory.getLogger(AggregationJobTest.class);
@BeforeAll
public static void beforeAll() throws IOException {
provenance = IOUtils.toString(AggregationJobTest.class.getResourceAsStream("/eu/dnetlib/dhp/collection/provenance.json"));
workingDir = Files.createTempDirectory(AggregationJobTest.class.getSimpleName());
log.info("using work dir {}", workingDir);
SparkConf conf = new SparkConf();
conf.setAppName(AggregationJobTest.class.getSimpleName());
conf.setMaster("local[*]");
conf.set("spark.driver.host", "localhost");
conf.set("hive.metastore.local", "true");
conf.set("spark.ui.enabled", "false");
conf.set("spark.sql.warehouse.dir", workingDir.toString());
conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString());
encoder = Encoders.bean(MetadataRecord.class);
spark = SparkSession
.builder()
.appName(AggregationJobTest.class.getSimpleName())
.config(conf)
.getOrCreate();
}
@AfterAll
public static void afterAll() throws IOException {
FileUtils.deleteDirectory(workingDir.toFile());
spark.stop();
}
@Test
@Order(1)
public void testGenerateNativeStoreSparkJobRefresh() throws Exception {
MDStoreVersion mdStoreV1 = prepareVersion("/eu/dnetlib/dhp/collection/mdStoreVersion_1.json");
FileUtils.forceMkdir(new File(mdStoreV1.getHdfsPath()));
IOUtils
.copy(
getClass().getResourceAsStream("/eu/dnetlib/dhp/collection/sequence_file"),
new FileOutputStream(mdStoreV1.getHdfsPath() + "/sequence_file"));
GenerateNativeStoreSparkJob
.main(
new String[]{
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-encoding", encoding,
"-dateOfCollection", dateOfCollection,
"-provenance", provenance,
"-xpath", xpath,
"-mdStoreVersion", OBJECT_MAPPER.writeValueAsString(mdStoreV1),
"-readMdStoreVersion", "",
"-workflowId", "abc"
});
verify(mdStoreV1);
}
@Test
@Order(2)
public void testGenerateNativeStoreSparkJobIncremental() throws Exception {
MDStoreVersion mdStoreV2 = prepareVersion("/eu/dnetlib/dhp/collection/mdStoreVersion_2.json");
FileUtils.forceMkdir(new File(mdStoreV2.getHdfsPath()));
IOUtils
.copy(
getClass().getResourceAsStream("/eu/dnetlib/dhp/collection/sequence_file"),
new FileOutputStream(mdStoreV2.getHdfsPath() + "/sequence_file"));
MDStoreVersion mdStoreV1 = prepareVersion("/eu/dnetlib/dhp/collection/mdStoreVersion_1.json");
GenerateNativeStoreSparkJob
.main(
new String[]{
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-encoding", encoding,
"-dateOfCollection", dateOfCollection,
"-provenance", provenance,
"-xpath", xpath,
"-mdStoreVersion", OBJECT_MAPPER.writeValueAsString(mdStoreV2),
"-readMdStoreVersion", OBJECT_MAPPER.writeValueAsString(mdStoreV1),
"-workflowId", "abc"
});
verify(mdStoreV2);
}
//@Test
@Order(3)
public void testTransformSparkJob() throws Exception {
MDStoreVersion mdStoreV2 = prepareVersion("/eu/dnetlib/dhp/collection/mdStoreVersion_2.json");
MDStoreVersion mdStoreCleanedVersion = prepareVersion("/eu/dnetlib/dhp/collection/mdStoreCleanedVersion.json");
TransformSparkJobNode.main(new String[]{
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-dateOfTransformation", dateOfCollection,
"-mdstoreInputVersion", OBJECT_MAPPER.writeValueAsString(mdStoreV2),
"-mdstoreOutputVersion", OBJECT_MAPPER.writeValueAsString(mdStoreCleanedVersion),
"-transformationPlugin", "XSLT_TRANSFORM",
"-isLookupUrl", "https://dev-openaire.d4science.org/is/services/isLookUp",
"-transformationRuleId", "183dde52-a69b-4db9-a07e-1ef2be105294_VHJhbnNmb3JtYXRpb25SdWxlRFNSZXNvdXJjZXMvVHJhbnNmb3JtYXRpb25SdWxlRFNSZXNvdXJjZVR5cGU="});
}
protected void verify(MDStoreVersion mdStoreVersion) throws IOException {
Assertions.assertTrue(new File(mdStoreVersion.getHdfsPath()).exists());
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
long seqFileSize = sc
.sequenceFile(mdStoreVersion.getHdfsPath() + "/sequence_file", IntWritable.class, Text.class)
.count();
final Dataset<MetadataRecord> mdstore = spark.read().load(mdStoreVersion.getHdfsPath() + "/store").as(encoder);
long mdStoreSize = mdstore.count();
long declaredSize = Long.parseLong(IOUtils.toString(new FileReader(mdStoreVersion.getHdfsPath() + "/size")));
Assertions.assertEquals(seqFileSize, declaredSize, "the size must be equal");
Assertions.assertEquals(seqFileSize, mdStoreSize, "the size must be equal");
long uniqueIds = mdstore
.map((MapFunction<MetadataRecord, String>) MetadataRecord::getId, Encoders.STRING())
.distinct()
.count();
Assertions.assertEquals(seqFileSize, uniqueIds, "the size must be equal");
}
private MDStoreVersion prepareVersion(String filename) throws IOException {
MDStoreVersion mdstore = OBJECT_MAPPER
.readValue(IOUtils.toString(getClass().getResource(filename)), MDStoreVersion.class);
mdstore.setHdfsPath(String.format(mdstore.getHdfsPath(), workingDir.toString()));
return mdstore;
}
}

@ -1,169 +0,0 @@
package eu.dnetlib.dhp.collection;
import java.io.File;
import java.io.FileOutputStream;
import java.io.FileReader;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Text;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoder;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession;
import org.junit.jupiter.api.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.data.mdstore.manager.common.model.MDStoreVersion;
import eu.dnetlib.dhp.model.mdstore.MetadataRecord;
@TestMethodOrder(MethodOrderer.OrderAnnotation.class)
public class GenerateNativeStoreSparkJobTest {
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static SparkSession spark;
private static Path workingDir;
private static Encoder<MetadataRecord> encoder;
private static final String encoding = "XML";
private static final String dateOfCollection = System.currentTimeMillis() + "";
private static final String xpath = "//*[local-name()='header']/*[local-name()='identifier']";
private static String provenance;
private static final Logger log = LoggerFactory.getLogger(GenerateNativeStoreSparkJobTest.class);
@BeforeAll
public static void beforeAll() throws IOException {
provenance = IOUtils.toString(GenerateNativeStoreSparkJobTest.class.getResourceAsStream("provenance.json"));
workingDir = Files.createTempDirectory(GenerateNativeStoreSparkJobTest.class.getSimpleName());
log.info("using work dir {}", workingDir);
SparkConf conf = new SparkConf();
conf.setAppName(GenerateNativeStoreSparkJobTest.class.getSimpleName());
conf.setMaster("local[*]");
conf.set("spark.driver.host", "localhost");
conf.set("hive.metastore.local", "true");
conf.set("spark.ui.enabled", "false");
conf.set("spark.sql.warehouse.dir", workingDir.toString());
conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString());
encoder = Encoders.bean(MetadataRecord.class);
spark = SparkSession
.builder()
.appName(GenerateNativeStoreSparkJobTest.class.getSimpleName())
.config(conf)
.getOrCreate();
}
@AfterAll
public static void afterAll() throws IOException {
FileUtils.deleteDirectory(workingDir.toFile());
spark.stop();
}
@Test
@Order(1)
public void testGenerateNativeStoreSparkJobRefresh() throws Exception {
MDStoreVersion mdStoreV1 = prepareVersion("mdStoreVersion_1.json");
FileUtils.forceMkdir(new File(mdStoreV1.getHdfsPath()));
IOUtils
.copy(
getClass().getResourceAsStream("sequence_file"),
new FileOutputStream(mdStoreV1.getHdfsPath() + "/sequence_file"));
GenerateNativeStoreSparkJob
.main(
new String[] {
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-encoding", encoding,
"-dateOfCollection", dateOfCollection,
"-provenance", provenance,
"-xpath", xpath,
"-mdStoreVersion", OBJECT_MAPPER.writeValueAsString(mdStoreV1),
"-readMdStoreVersion", "",
"-workflowId", "abc"
});
verify(mdStoreV1);
}
@Test
@Order(2)
public void testGenerateNativeStoreSparkJobIncremental() throws Exception {
MDStoreVersion mdStoreV2 = prepareVersion("mdStoreVersion_2.json");
FileUtils.forceMkdir(new File(mdStoreV2.getHdfsPath()));
IOUtils
.copy(
getClass().getResourceAsStream("sequence_file"),
new FileOutputStream(mdStoreV2.getHdfsPath() + "/sequence_file"));
MDStoreVersion mdStoreV1 = prepareVersion("mdStoreVersion_1.json");
GenerateNativeStoreSparkJob
.main(
new String[] {
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-encoding", encoding,
"-dateOfCollection", dateOfCollection,
"-provenance", provenance,
"-xpath", xpath,
"-mdStoreVersion", OBJECT_MAPPER.writeValueAsString(mdStoreV2),
"-readMdStoreVersion", OBJECT_MAPPER.writeValueAsString(mdStoreV1),
"-workflowId", "abc"
});
verify(mdStoreV2);
}
protected void verify(MDStoreVersion mdStoreVersion) throws IOException {
Assertions.assertTrue(new File(mdStoreVersion.getHdfsPath()).exists());
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
long seqFileSize = sc
.sequenceFile(mdStoreVersion.getHdfsPath() + "/sequence_file", IntWritable.class, Text.class)
.count();
final Dataset<MetadataRecord> mdstore = spark.read().load(mdStoreVersion.getHdfsPath() + "/store").as(encoder);
long mdStoreSize = mdstore.count();
long declaredSize = Long.parseLong(IOUtils.toString(new FileReader(mdStoreVersion.getHdfsPath() + "/size")));
Assertions.assertEquals(seqFileSize, declaredSize, "the size must be equal");
Assertions.assertEquals(seqFileSize, mdStoreSize, "the size must be equal");
long uniqueIds = mdstore
.map((MapFunction<MetadataRecord, String>) MetadataRecord::getId, Encoders.STRING())
.distinct()
.count();
Assertions.assertEquals(seqFileSize, uniqueIds, "the size must be equal");
}
private MDStoreVersion prepareVersion(String filename) throws IOException {
MDStoreVersion mdstore = OBJECT_MAPPER
.readValue(IOUtils.toString(getClass().getResource(filename)), MDStoreVersion.class);
mdstore.setHdfsPath(String.format(mdstore.getHdfsPath(), workingDir.toString()));
return mdstore;
}
}

@ -38,6 +38,7 @@ import eu.dnetlib.dhp.collection.CollectionJobTest;
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
import eu.dnetlib.dhp.model.mdstore.MetadataRecord;
import eu.dnetlib.dhp.transformation.xslt.XSLTTransformationFunction;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
@ -74,6 +75,9 @@ public class TransformationJobTest {
spark.stop();
}
@Test
@DisplayName("Test Transform Single XML using XSLTTransformator")
public void testTransformSaxonHE() throws Exception {

@ -0,0 +1,9 @@
{
"id":"md-cleaned",
"mdstore":"md-cleaned",
"writing":false,
"readCount":1,
"lastUpdate":1612187563099,
"size":71,
"hdfsPath":"%s/mdstore/md-cleaned"
}
Loading…
Cancel
Save