|
59 | 59 | import org.apache.pulsar.client.api.Producer;
|
60 | 60 | import org.apache.pulsar.client.api.PulsarClient;
|
61 | 61 | import org.apache.pulsar.client.api.Schema;
|
| 62 | +import org.apache.pulsar.client.api.schema.GenericRecord; |
| 63 | +import org.apache.pulsar.client.api.schema.SchemaDefinition; |
62 | 64 | import org.apache.pulsar.client.impl.auth.AuthenticationTls;
|
63 | 65 | import org.apache.pulsar.common.functions.ConsumerConfig;
|
64 | 66 | import org.apache.pulsar.common.functions.FunctionConfig;
|
|
72 | 74 | import org.apache.pulsar.common.util.FutureUtil;
|
73 | 75 | import org.apache.pulsar.common.util.ObjectMapperFactory;
|
74 | 76 | import org.apache.pulsar.functions.LocalRunner;
|
| 77 | +import org.apache.pulsar.functions.api.examples.pojo.AvroTestObject; |
75 | 78 | import org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactory;
|
76 | 79 | import org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactoryConfig;
|
77 | 80 | import org.apache.pulsar.io.datagenerator.DataGeneratorPrintSink;
|
@@ -479,11 +482,134 @@ private void testE2EPulsarFunctionLocalRun(String jarFilePathUrl) throws Excepti
|
479 | 482 | }
|
480 | 483 | }
|
481 | 484 |
|
| 485 | + public void testAvroFunctionLocalRun(String jarFilePathUrl) throws Exception { |
| 486 | + |
| 487 | + final String namespacePortion = "io"; |
| 488 | + final String replNamespace = tenant + "/" + namespacePortion; |
| 489 | + final String sourceTopic = "persistent://" + replNamespace + "/my-topic1"; |
| 490 | + final String sinkTopic = "persistent://" + replNamespace + "/output"; |
| 491 | + final String propertyKey = "key"; |
| 492 | + final String propertyValue = "value"; |
| 493 | + final String functionName = "PulsarFunction-test"; |
| 494 | + final String subscriptionName = "test-sub"; |
| 495 | + admin.namespaces().createNamespace(replNamespace); |
| 496 | + Set<String> clusters = Sets.newHashSet(Lists.newArrayList(CLUSTER)); |
| 497 | + admin.namespaces().setNamespaceReplicationClusters(replNamespace, clusters); |
| 498 | + |
| 499 | + |
| 500 | + Schema schema = Schema.AVRO(SchemaDefinition.builder() |
| 501 | + .withAlwaysAllowNull(true) |
| 502 | + .withJSR310ConversionEnabled(true) |
| 503 | + .withPojo(AvroTestObject.class).build()); |
| 504 | + //use AVRO schema |
| 505 | + admin.schemas().createSchema(sourceTopic, schema.getSchemaInfo()); |
| 506 | + |
| 507 | + //produce message to sourceTopic |
| 508 | + Producer<AvroTestObject> producer = pulsarClient.newProducer(schema).topic(sourceTopic).create(); |
| 509 | + //consume message from sinkTopic |
| 510 | + Consumer<GenericRecord> consumer = pulsarClient.newConsumer(Schema.AUTO_CONSUME()).topic(sinkTopic).subscriptionName("sub").subscribe(); |
| 511 | + |
| 512 | + FunctionConfig functionConfig = createFunctionConfig(tenant, namespacePortion, functionName, |
| 513 | + sourceTopic, sinkTopic, subscriptionName); |
| 514 | + //set jsr310ConversionEnabled、alwaysAllowNull |
| 515 | + Map<String,String> schemaInput = new HashMap<>(); |
| 516 | + schemaInput.put(sourceTopic, "{\"schemaType\":\"AVRO\",\"schemaProperties\":{\"__jsr310ConversionEnabled\":\"true\",\"__alwaysAllowNull\":\"true\"}}"); |
| 517 | + Map<String, String> schemaOutput = new HashMap<>(); |
| 518 | + schemaOutput.put(sinkTopic, "{\"schemaType\":\"AVRO\",\"schemaProperties\":{\"__jsr310ConversionEnabled\":\"true\",\"__alwaysAllowNull\":\"true\"}}"); |
| 519 | + |
| 520 | + functionConfig.setCustomSchemaInputs(schemaInput); |
| 521 | + functionConfig.setCustomSchemaOutputs(schemaOutput); |
| 522 | + functionConfig.setProcessingGuarantees(FunctionConfig.ProcessingGuarantees.ATLEAST_ONCE); |
| 523 | + if (jarFilePathUrl == null) { |
| 524 | + functionConfig.setClassName("org.apache.pulsar.functions.api.examples.AvroSchemaTestFunction"); |
| 525 | + } else { |
| 526 | + functionConfig.setJar(jarFilePathUrl); |
| 527 | + } |
| 528 | + |
| 529 | + LocalRunner localRunner = LocalRunner.builder() |
| 530 | + .functionConfig(functionConfig) |
| 531 | + .clientAuthPlugin(AuthenticationTls.class.getName()) |
| 532 | + .clientAuthParams(String.format("tlsCertFile:%s,tlsKeyFile:%s", TLS_CLIENT_CERT_FILE_PATH, TLS_CLIENT_KEY_FILE_PATH)) |
| 533 | + .useTls(true) |
| 534 | + .tlsTrustCertFilePath(TLS_TRUST_CERT_FILE_PATH) |
| 535 | + .tlsAllowInsecureConnection(true) |
| 536 | + .tlsHostNameVerificationEnabled(false) |
| 537 | + .brokerServiceUrl(pulsar.getBrokerServiceUrlTls()).build(); |
| 538 | + localRunner.start(false); |
| 539 | + |
| 540 | + retryStrategically((test) -> { |
| 541 | + try { |
| 542 | + TopicStats stats = admin.topics().getStats(sourceTopic); |
| 543 | + return stats.subscriptions.get(subscriptionName) != null |
| 544 | + && !stats.subscriptions.get(subscriptionName).consumers.isEmpty(); |
| 545 | + } catch (PulsarAdminException e) { |
| 546 | + return false; |
| 547 | + } |
| 548 | + }, 50, 150); |
| 549 | + |
| 550 | + int totalMsgs = 5; |
| 551 | + for (int i = 0; i < totalMsgs; i++) { |
| 552 | + AvroTestObject avroTestObject = new AvroTestObject(); |
| 553 | + avroTestObject.setBaseValue(i); |
| 554 | + producer.newMessage().property(propertyKey, propertyValue) |
| 555 | + .value(avroTestObject).send(); |
| 556 | + } |
| 557 | + |
| 558 | + //consume message from sinkTopic |
| 559 | + for (int i = 0; i < totalMsgs; i++) { |
| 560 | + Message<GenericRecord> msg = consumer.receive(5, TimeUnit.SECONDS); |
| 561 | + String receivedPropertyValue = msg.getProperty(propertyKey); |
| 562 | + assertEquals(propertyValue, receivedPropertyValue); |
| 563 | + assertEquals(msg.getValue().getField("baseValue"), 10 + i); |
| 564 | + consumer.acknowledge(msg); |
| 565 | + } |
| 566 | + |
| 567 | + // validate pulsar-sink consumer has consumed all messages |
| 568 | + assertNotEquals(admin.topics().getStats(sinkTopic).subscriptions.values().iterator().next().unackedMessages, 0); |
| 569 | + localRunner.stop(); |
| 570 | + |
| 571 | + retryStrategically((test) -> { |
| 572 | + try { |
| 573 | + TopicStats topicStats = admin.topics().getStats(sourceTopic); |
| 574 | + return topicStats.subscriptions.get(subscriptionName) != null |
| 575 | + && topicStats.subscriptions.get(subscriptionName).consumers.isEmpty(); |
| 576 | + } catch (PulsarAdminException e) { |
| 577 | + return false; |
| 578 | + } |
| 579 | + }, 20, 150); |
| 580 | + |
| 581 | + //change the schema ,the function should not run, resulting in no messages to consume |
| 582 | + schemaInput.put(sourceTopic, "{\"schemaType\":\"AVRO\",\"schemaProperties\":{\"__jsr310ConversionEnabled\":\"false\",\"__alwaysAllowNull\":\"false\"}}"); |
| 583 | + localRunner = LocalRunner.builder() |
| 584 | + .functionConfig(functionConfig) |
| 585 | + .clientAuthPlugin(AuthenticationTls.class.getName()) |
| 586 | + .clientAuthParams(String.format("tlsCertFile:%s,tlsKeyFile:%s", TLS_CLIENT_CERT_FILE_PATH, TLS_CLIENT_KEY_FILE_PATH)) |
| 587 | + .useTls(true) |
| 588 | + .tlsTrustCertFilePath(TLS_TRUST_CERT_FILE_PATH) |
| 589 | + .tlsAllowInsecureConnection(true) |
| 590 | + .tlsHostNameVerificationEnabled(false) |
| 591 | + .brokerServiceUrl(pulsar.getBrokerServiceUrlTls()).build(); |
| 592 | + localRunner.start(false); |
| 593 | + |
| 594 | + producer.newMessage().property(propertyKey, propertyValue).value(new AvroTestObject()).send(); |
| 595 | + Message<GenericRecord> msg = consumer.receive(2, TimeUnit.SECONDS); |
| 596 | + assertEquals(msg, null); |
| 597 | + |
| 598 | + producer.close(); |
| 599 | + consumer.close(); |
| 600 | + localRunner.stop(); |
| 601 | + } |
| 602 | + |
482 | 603 | @Test(timeOut = 20000)
|
483 | 604 | public void testE2EPulsarFunctionLocalRun() throws Exception {
|
484 | 605 | testE2EPulsarFunctionLocalRun(null);
|
485 | 606 | }
|
486 | 607 |
|
| 608 | + @Test(timeOut = 30000) |
| 609 | + public void testAvroFunctionLocalRun() throws Exception { |
| 610 | + testAvroFunctionLocalRun(null); |
| 611 | + } |
| 612 | + |
487 | 613 | @Test(timeOut = 20000)
|
488 | 614 | public void testE2EPulsarFunctionLocalRunWithJar() throws Exception {
|
489 | 615 | String jarFilePathUrl = Utils.FILE + ":" + getClass().getClassLoader().getResource("pulsar-functions-api-examples.jar").getFile();
|
|
0 commit comments